mirror of https://github.com/apache/nifi.git
NIFI-8113 Adding persistent status history repository backed by embedded QuestDB
Signed-off-by: Mark Payne <markap14@hotmail.com>
This commit is contained in:
parent
82eedea659
commit
7730777d66
|
@ -1956,6 +1956,11 @@ The following binary components are provided under the Apache Software License v
|
|||
Zip4j 2.6.3.
|
||||
Copyright 2020 Srikanth Reddy Lingala
|
||||
|
||||
(ASLv2) QuestDB (org.questdb:core:core-4.2.1.jar - https://github.com/questdb/questdb)
|
||||
The following NOTICE information applies:
|
||||
Copyright (c) 2014-2019 Appsicle
|
||||
Copyright (c) 2019-2020 QuestDB
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.1
|
||||
************************
|
||||
|
|
|
@ -136,10 +136,15 @@ public abstract class NiFiProperties {
|
|||
public static final String PROVENANCE_REPO_ENCRYPTION_KEY_PROVIDER_LOCATION = "nifi.provenance.repository.encryption.key.provider.location";
|
||||
public static final String PROVENANCE_REPO_DEBUG_FREQUENCY = "nifi.provenance.repository.debug.frequency";
|
||||
|
||||
// component status repository properties
|
||||
// status repository properties
|
||||
public static final String COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION = "nifi.components.status.repository.implementation";
|
||||
public static final String COMPONENT_STATUS_SNAPSHOT_FREQUENCY = "nifi.components.status.snapshot.frequency";
|
||||
|
||||
// questdb status storage properties
|
||||
public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_NODE_DAYS = "nifi.status.repository.questdb.persist.node.days";
|
||||
public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_COMPONENT_DAYS = "nifi.status.repository.questdb.persist.component.days";
|
||||
public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_LOCATION = "nifi.status.repository.questdb.persist.location";
|
||||
|
||||
// security properties
|
||||
public static final String SECURITY_KEYSTORE = "nifi.security.keystore";
|
||||
public static final String SECURITY_KEYSTORE_TYPE = "nifi.security.keystoreType";
|
||||
|
@ -366,6 +371,11 @@ public abstract class NiFiProperties {
|
|||
public static final String DEFAULT_ANALYTICS_CONNECTION_SCORE_NAME = "rSquared";
|
||||
public static final double DEFAULT_ANALYTICS_CONNECTION_SCORE_THRESHOLD = .90;
|
||||
|
||||
// Status repository defaults
|
||||
public static final int DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_NODE_DAYS = 14;
|
||||
public static final int DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_COMPONENT_DAYS = 3;
|
||||
public static final String DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_LOCATION = "./status_repository";
|
||||
|
||||
/**
|
||||
* Retrieves the property value for the given property key.
|
||||
*
|
||||
|
@ -1855,6 +1865,15 @@ public abstract class NiFiProperties {
|
|||
return getProperty(BACKPRESSURE_SIZE, DEFAULT_BACKPRESSURE_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the directory where the QuestDB based status repository is expected to work within.
|
||||
*
|
||||
* @return Path object pointing to the database's folder.
|
||||
*/
|
||||
public Path getQuestDbStatusRepositoryPath() {
|
||||
return Paths.get(getProperty(STATUS_REPOSITORY_QUESTDB_PERSIST_LOCATION, DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_LOCATION));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of NiFiProperties. This should likely not be called
|
||||
* by any classes outside of the NiFi framework but can be useful by the
|
||||
|
|
|
@ -3089,10 +3089,21 @@ Providing three total locations, including `nifi.provenance.repository.directory
|
|||
|====
|
||||
|
||||
|
||||
=== Component Status Repository
|
||||
=== Status History Repository
|
||||
|
||||
The Component Status Repository contains the information for the Component Status History tool in the User Interface. These
|
||||
properties govern how that tool works.
|
||||
The Status History Repository contains the information for the Component Status History and the Node Status History tools in
|
||||
the User Interface. The following properties govern how these tools work.
|
||||
|
||||
|====
|
||||
|*Property*|*Description*
|
||||
|`nifi.components.status.repository.implementation`|The Status History Repository implementation. The default value is `VolatileComponentStatusRepository`. `EmbeddedQuestDbStatusHistoryRepository` is also supported.
|
||||
|`nifi.components.status.snapshot.frequency`|This value indicates how often to present a snapshot of the components' status history. The default value is `1 min`.
|
||||
|====
|
||||
|
||||
==== In memory repository
|
||||
|
||||
In case the value of the property `nifi.components.status.repository.implementation` is `VolatileComponentStatusRepository`, the
|
||||
status history data will be stored in memory. In case the application stops, all gathered information will be lost.
|
||||
|
||||
The `buffer.size` and `snapshot.frequency` work together to determine the amount of historical data to retain. As an example to
|
||||
configure two days worth of historical data with a data point snapshot occurring every 5 minutes you would configure
|
||||
|
@ -3102,11 +3113,20 @@ of 576.
|
|||
|
||||
|====
|
||||
|*Property*|*Description*
|
||||
|`nifi.components.status.repository.implementation`|The Component Status Repository implementation. The default value is `org.apache.nifi.controller.status.history.VolatileComponentStatusRepository` and should not be changed.
|
||||
|`nifi.components.status.repository.buffer.size`|Specifies the buffer size for the Component Status Repository. The default value is `1440`.
|
||||
|`nifi.components.status.snapshot.frequency`|This value indicates how often to present a snapshot of the components' status history. The default value is `1 min`.
|
||||
|`nifi.components.status.repository.buffer.size`|Specifies the buffer size for the Status History Repository. The default value is `1440`.
|
||||
|====
|
||||
|
||||
==== Persistent repository
|
||||
|
||||
In case the value of the property `nifi.components.status.repository.implementation` is `EmbeddedQuestDbStatusHistoryRepository`, the
|
||||
status history data will be stored to the disk in a persistent manner. Data will be kept between restarts.
|
||||
|
||||
|====
|
||||
|*Property*|*Description*
|
||||
|`nifi.status.repository.questdb.persist.node.days`|The number of days the node status data will be kept. The default values is `14`.
|
||||
|`nifi.status.repository.questdb.persist.component.days`|The number of days the component status data will be kept. The default value is `3`.
|
||||
|`nifi.status.repository.questdb.persist.location`|The location of the persistent Status History Repository. The default value is `./status_repository`.
|
||||
|====
|
||||
|
||||
[[site_to_site_properties]]
|
||||
=== Site to Site Properties
|
||||
|
|
|
@ -16,17 +16,17 @@
|
|||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.apache.nifi.controller.status.ProcessGroupStatus;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A repository for storing and retrieving components' historical status
|
||||
* A repository for storing and retrieving node and components' historical status
|
||||
* information
|
||||
*/
|
||||
public interface ComponentStatusRepository {
|
||||
public interface StatusHistoryRepository {
|
||||
|
||||
String COMPONENT_DETAIL_ID = "Id";
|
||||
String COMPONENT_DETAIL_GROUP_ID = "Group Id";
|
||||
|
@ -36,15 +36,6 @@ public interface ComponentStatusRepository {
|
|||
String COMPONENT_DETAIL_DESTINATION_NAME = "Destination Name";
|
||||
String COMPONENT_DETAIL_URI = "Uri";
|
||||
|
||||
/**
|
||||
* Captures the status information provided in the given report
|
||||
*
|
||||
* @param nodeStatus status of the node
|
||||
* @param rootGroupStatus status of root group and it's content
|
||||
* @param garbageCollectionStatus status of garbage collection
|
||||
*/
|
||||
void capture(NodeStatus nodeStatus, ProcessGroupStatus rootGroupStatus, List<GarbageCollectionStatus> garbageCollectionStatus);
|
||||
|
||||
/**
|
||||
* Captures the status information provided in the given report, providing a
|
||||
* timestamp that indicates the time at which the status report was
|
||||
|
@ -57,11 +48,6 @@ public interface ComponentStatusRepository {
|
|||
*/
|
||||
void capture(NodeStatus nodeStatus, ProcessGroupStatus rootGroupStatus, List<GarbageCollectionStatus> garbageCollectionStatus, Date timestamp);
|
||||
|
||||
/**
|
||||
* @return the Date at which the latest capture was performed
|
||||
*/
|
||||
Date getLastCaptureDate();
|
||||
|
||||
/**
|
||||
* @param connectionId the ID of the Connection for which the Status is
|
||||
* desired
|
||||
|
@ -135,12 +121,42 @@ public interface ComponentStatusRepository {
|
|||
StatusHistory getRemoteProcessGroupStatusHistory(String remoteGroupId, Date start, Date end, int preferredDataPoints);
|
||||
|
||||
/**
|
||||
* Returns the status history of the actual node
|
||||
* Returns the status history of the actual node.
|
||||
*
|
||||
* @param start the earliest date for which status information should be
|
||||
* returned; if <code>null</code>, the start date should be assumed to be
|
||||
* the beginning of time
|
||||
* @param end the latest date for which status information should be
|
||||
* returned; if <code>null</code>, the end date should be assumed to be the
|
||||
* current time
|
||||
*
|
||||
* @return a {@link StatusHistory} that provides the status information
|
||||
* about the NiFi node from the period stored in the status repository.
|
||||
* about the NiFi node within the specified time range.
|
||||
*/
|
||||
StatusHistory getNodeStatusHistory();
|
||||
StatusHistory getNodeStatusHistory(Date start, Date end);
|
||||
|
||||
/**
|
||||
* Returns the status history of the garbage collection.
|
||||
*
|
||||
* @param start the earliest date for which status information should be
|
||||
* returned; if <code>null</code>, the start date should be assumed to be
|
||||
* the beginning of time
|
||||
* @param end the latest date for which status information should be
|
||||
* returned; if <code>null</code>, the end date should be assumed to be the
|
||||
* current time
|
||||
*
|
||||
* @return a {@link GarbageCollectionHistory} that provides the status information
|
||||
* about the garbage collection of the given node within the specified time range
|
||||
*/
|
||||
GarbageCollectionHistory getGarbageCollectionHistory(Date start, Date end);
|
||||
|
||||
/**
|
||||
* Starts necessary resources needed for the repository.
|
||||
*/
|
||||
void start();
|
||||
|
||||
/**
|
||||
* Stops the resources used by the repository.
|
||||
*/
|
||||
void shutdown();
|
||||
}
|
|
@ -206,6 +206,10 @@ The following binary components are provided under the Apache Software License v
|
|||
Bytes Utility Library 1.3.0
|
||||
Copyright 2017 Patrick Favre-Bulle
|
||||
|
||||
(ASLv2) QuestDB (org.questdb:core:core-4.2.1.jar - https://github.com/questdb/questdb)
|
||||
The following NOTICE information applies:
|
||||
Copyright (c) 2014-2019 Appsicle
|
||||
Copyright (c) 2019-2020 QuestDB
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.1
|
||||
|
|
|
@ -139,6 +139,11 @@
|
|||
<artifactId>commons-math3</artifactId>
|
||||
<version>3.6.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.questdb</groupId>
|
||||
<artifactId>core</artifactId>
|
||||
<version>4.2.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-data-provenance-utils</artifactId>
|
||||
|
|
|
@ -121,10 +121,10 @@ import org.apache.nifi.controller.status.analytics.CachingConnectionStatusAnalyt
|
|||
import org.apache.nifi.controller.status.analytics.ConnectionStatusAnalytics;
|
||||
import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine;
|
||||
import org.apache.nifi.controller.status.analytics.StatusAnalyticsModelMapFactory;
|
||||
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
|
||||
import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
|
||||
import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
|
||||
import org.apache.nifi.controller.status.history.StandardGarbageCollectionStatus;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryUtil;
|
||||
import org.apache.nifi.controller.tasks.ExpireFlowFiles;
|
||||
import org.apache.nifi.diagnostics.StorageUsage;
|
||||
|
@ -239,7 +239,6 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
public static final String GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.seconds";
|
||||
public static final long DEFAULT_GRACEFUL_SHUTDOWN_SECONDS = 10;
|
||||
|
||||
|
||||
// default properties for scaling the positions of components from pre-1.0 flow encoding versions.
|
||||
public static final double DEFAULT_POSITION_SCALE_FACTOR_X = 1.5;
|
||||
public static final double DEFAULT_POSITION_SCALE_FACTOR_Y = 1.34;
|
||||
|
@ -269,7 +268,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
private final Authorizer authorizer;
|
||||
private final AuditService auditService;
|
||||
private final EventDrivenWorkerQueue eventDrivenWorkerQueue;
|
||||
private final ComponentStatusRepository componentStatusRepository;
|
||||
private final StatusHistoryRepository statusHistoryRepository;
|
||||
private final StateManagerProvider stateManagerProvider;
|
||||
private final long systemStartTime = System.currentTimeMillis(); // time at which the node was started
|
||||
private final VariableRegistry variableRegistry;
|
||||
|
@ -620,7 +619,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
zooKeeperStateServer = null;
|
||||
}
|
||||
|
||||
componentStatusRepository = createComponentStatusRepository();
|
||||
statusHistoryRepository = createStatusHistoryRepository();
|
||||
|
||||
final boolean analyticsEnabled = Boolean.parseBoolean(nifiProperties.getProperty(NiFiProperties.ANALYTICS_PREDICTION_ENABLED, NiFiProperties.DEFAULT_ANALYTICS_PREDICTION_ENABLED));
|
||||
|
||||
|
@ -664,7 +663,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
|
||||
StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory = new StatusAnalyticsModelMapFactory(extensionManager, nifiProperties);
|
||||
|
||||
analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, statusAnalyticsModelMapFactory,
|
||||
analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusHistoryRepository, statusAnalyticsModelMapFactory,
|
||||
predictionIntervalMillis, queryIntervalMillis, modelScoreName, modelScoreThreshold);
|
||||
|
||||
timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
|
||||
|
@ -694,7 +693,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
componentStatusRepository.capture(getNodeStatusSnapshot(), eventAccess.getControllerStatus(), getGarbageCollectionStatus());
|
||||
statusHistoryRepository.capture(getNodeStatusSnapshot(), eventAccess.getControllerStatus(), getGarbageCollectionStatus(), new Date());
|
||||
} catch (final Exception e) {
|
||||
LOG.error("Failed to capture component stats for Stats History", e);
|
||||
}
|
||||
|
@ -1124,21 +1123,22 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
}
|
||||
}
|
||||
|
||||
private ComponentStatusRepository createComponentStatusRepository() {
|
||||
private StatusHistoryRepository createStatusHistoryRepository() {
|
||||
final String implementationClassName = nifiProperties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
|
||||
if (implementationClassName == null) {
|
||||
throw new RuntimeException("Cannot create Component Status Repository because the NiFi Properties is missing the following property: "
|
||||
throw new RuntimeException("Cannot create Status History Repository because the NiFi Properties is missing the following property: "
|
||||
+ NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
|
||||
}
|
||||
|
||||
try {
|
||||
return NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, ComponentStatusRepository.class, nifiProperties);
|
||||
final StatusHistoryRepository repository = NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, StatusHistoryRepository.class, nifiProperties);
|
||||
repository.start();
|
||||
return repository;
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public KerberosConfig createKerberosConfig(final NiFiProperties nifiProperties) {
|
||||
final String principal = nifiProperties.getKerberosServicePrincipal();
|
||||
final String keytabLocation = nifiProperties.getKerberosServiceKeytabLocation();
|
||||
|
@ -1332,6 +1332,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (statusHistoryRepository != null) {
|
||||
statusHistoryRepository.shutdown();
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock("shutdown");
|
||||
}
|
||||
|
@ -1693,7 +1697,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
}
|
||||
|
||||
public GarbageCollectionHistory getGarbageCollectionHistory() {
|
||||
return componentStatusRepository.getGarbageCollectionHistory(new Date(0L), new Date());
|
||||
return statusHistoryRepository.getGarbageCollectionHistory(new Date(0L), new Date());
|
||||
}
|
||||
|
||||
|
||||
|
@ -2967,7 +2971,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
}
|
||||
|
||||
public StatusHistoryDTO getConnectionStatusHistory(final String connectionId, final Date startTime, final Date endTime, final int preferredDataPoints) {
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(componentStatusRepository.getConnectionStatusHistory(connectionId, startTime, endTime, preferredDataPoints));
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(statusHistoryRepository.getConnectionStatusHistory(connectionId, startTime, endTime, preferredDataPoints));
|
||||
}
|
||||
|
||||
public StatusHistoryDTO getProcessorStatusHistory(final String processorId, final boolean includeCounters) {
|
||||
|
@ -2975,7 +2979,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
}
|
||||
|
||||
public StatusHistoryDTO getProcessorStatusHistory(final String processorId, final Date startTime, final Date endTime, final int preferredDataPoints, final boolean includeCounters) {
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(componentStatusRepository.getProcessorStatusHistory(processorId, startTime, endTime, preferredDataPoints, includeCounters));
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(statusHistoryRepository.getProcessorStatusHistory(processorId, startTime, endTime, preferredDataPoints, includeCounters));
|
||||
}
|
||||
|
||||
public StatusHistoryDTO getProcessGroupStatusHistory(final String processGroupId) {
|
||||
|
@ -2983,7 +2987,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
}
|
||||
|
||||
public StatusHistoryDTO getProcessGroupStatusHistory(final String processGroupId, final Date startTime, final Date endTime, final int preferredDataPoints) {
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(componentStatusRepository.getProcessGroupStatusHistory(processGroupId, startTime, endTime, preferredDataPoints));
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(statusHistoryRepository.getProcessGroupStatusHistory(processGroupId, startTime, endTime, preferredDataPoints));
|
||||
}
|
||||
|
||||
public StatusHistoryDTO getRemoteProcessGroupStatusHistory(final String remoteGroupId) {
|
||||
|
@ -2991,11 +2995,11 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
}
|
||||
|
||||
public StatusHistoryDTO getRemoteProcessGroupStatusHistory(final String remoteGroupId, final Date startTime, final Date endTime, final int preferredDataPoints) {
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(componentStatusRepository.getRemoteProcessGroupStatusHistory(remoteGroupId, startTime, endTime, preferredDataPoints));
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(statusHistoryRepository.getRemoteProcessGroupStatusHistory(remoteGroupId, startTime, endTime, preferredDataPoints));
|
||||
}
|
||||
|
||||
public StatusHistoryDTO getNodeStatusHistory() {
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(componentStatusRepository.getNodeStatusHistory());
|
||||
return StatusHistoryUtil.createStatusHistoryDTO(statusHistoryRepository.getNodeStatusHistory(null, null));
|
||||
}
|
||||
|
||||
private NodeStatus getNodeStatusSnapshot() {
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.nifi.controller.status.analytics;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.controller.flow.FlowManager;
|
||||
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -35,7 +35,7 @@ public class CachingConnectionStatusAnalyticsEngine extends ConnectionStatusAnal
|
|||
private volatile Cache<String, StatusAnalytics> cache;
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CachingConnectionStatusAnalyticsEngine.class);
|
||||
|
||||
public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository,
|
||||
public CachingConnectionStatusAnalyticsEngine(FlowManager flowManager, StatusHistoryRepository statusRepository,
|
||||
StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory,
|
||||
long predictionIntervalMillis, long queryIntervalMillis, String scoreName, double scoreThreshold) {
|
||||
|
||||
|
|
|
@ -30,8 +30,8 @@ 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.RepositoryStatusReport;
|
||||
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusHistory;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.util.Tuple;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -48,7 +48,7 @@ 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 StatusHistoryRepository statusHistoryRepository;
|
||||
private final String connectionIdentifier;
|
||||
private final FlowManager flowManager;
|
||||
private final Boolean supportOnlineLearning;
|
||||
|
@ -67,9 +67,9 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
|
|||
private static String NEXT_INTERVAL_PERCENTAGE_USE_BYTES = "nextIntervalPercentageUseBytes";
|
||||
private static String INTERVAL_TIME_MILLIS = "intervalTimeMillis";
|
||||
|
||||
public ConnectionStatusAnalytics(ComponentStatusRepository componentStatusRepository, FlowManager flowManager,
|
||||
public ConnectionStatusAnalytics(StatusHistoryRepository statusHistoryRepository, FlowManager flowManager,
|
||||
Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap, String connectionIdentifier, Boolean supportOnlineLearning) {
|
||||
this.componentStatusRepository = componentStatusRepository;
|
||||
this.statusHistoryRepository = statusHistoryRepository;
|
||||
this.flowManager = flowManager;
|
||||
this.modelMap = modelMap;
|
||||
this.connectionIdentifier = connectionIdentifier;
|
||||
|
@ -93,7 +93,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics {
|
|||
|
||||
StatusAnalyticsModel model = modelFunction.getKey();
|
||||
StatusMetricExtractFunction extract = modelFunction.getValue();
|
||||
StatusHistory statusHistory = componentStatusRepository.getConnectionStatusHistory(connectionIdentifier, queryWindow.getStartDateTime(), queryWindow.getEndDateTime(), Integer.MAX_VALUE);
|
||||
StatusHistory statusHistory = statusHistoryRepository.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]);
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.nifi.controller.status.analytics;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.nifi.controller.flow.FlowManager;
|
||||
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
import org.apache.nifi.util.Tuple;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
|
|||
*/
|
||||
public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ConnectionStatusAnalyticsEngine.class);
|
||||
protected final ComponentStatusRepository statusRepository;
|
||||
protected final StatusHistoryRepository statusRepository;
|
||||
protected final FlowManager flowManager;
|
||||
protected final StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory;
|
||||
protected final long predictionIntervalMillis;
|
||||
|
@ -39,7 +39,7 @@ public class ConnectionStatusAnalyticsEngine implements StatusAnalyticsEngine {
|
|||
protected final String scoreName;
|
||||
protected final double scoreThreshold;
|
||||
|
||||
public ConnectionStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository,
|
||||
public ConnectionStatusAnalyticsEngine(FlowManager flowManager, StatusHistoryRepository statusRepository,
|
||||
StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory, long predictionIntervalMillis,
|
||||
long queryIntervalMillis, String scoreName, double scoreThreshold) {
|
||||
this.flowManager = flowManager;
|
||||
|
|
|
@ -106,18 +106,18 @@ public class ComponentDetails {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns a {@Link Map} whose keys are those values defined by {@link ComponentStatusRepository#COMPONENT_DETAIL_GROUP_ID ComponentStatusRepository.COMPONENT_DETAIL_*}
|
||||
* Returns a {@Link Map} whose keys are those values defined by {@link StatusHistoryRepository#COMPONENT_DETAIL_GROUP_ID ComponentStatusRepository.COMPONENT_DETAIL_*}
|
||||
* and values are the values that are populated for this ComponentDetails object.
|
||||
*/
|
||||
public Map<String, String> toMap() {
|
||||
final Map<String, String> map = new HashMap<>();
|
||||
addToMap(map, ComponentStatusRepository.COMPONENT_DETAIL_ID, componentId);
|
||||
addToMap(map, ComponentStatusRepository.COMPONENT_DETAIL_GROUP_ID, groupId);
|
||||
addToMap(map, ComponentStatusRepository.COMPONENT_DETAIL_NAME, componentName);
|
||||
addToMap(map, ComponentStatusRepository.COMPONENT_DETAIL_TYPE, componentType);
|
||||
addToMap(map, ComponentStatusRepository.COMPONENT_DETAIL_SOURCE_NAME, sourceName);
|
||||
addToMap(map, ComponentStatusRepository.COMPONENT_DETAIL_DESTINATION_NAME, destinationName);
|
||||
addToMap(map, ComponentStatusRepository.COMPONENT_DETAIL_URI, targetUri);
|
||||
addToMap(map, StatusHistoryRepository.COMPONENT_DETAIL_ID, componentId);
|
||||
addToMap(map, StatusHistoryRepository.COMPONENT_DETAIL_GROUP_ID, groupId);
|
||||
addToMap(map, StatusHistoryRepository.COMPONENT_DETAIL_NAME, componentName);
|
||||
addToMap(map, StatusHistoryRepository.COMPONENT_DETAIL_TYPE, componentType);
|
||||
addToMap(map, StatusHistoryRepository.COMPONENT_DETAIL_SOURCE_NAME, sourceName);
|
||||
addToMap(map, StatusHistoryRepository.COMPONENT_DETAIL_DESTINATION_NAME, destinationName);
|
||||
addToMap(map, StatusHistoryRepository.COMPONENT_DETAIL_URI, targetUri);
|
||||
return map;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Stores and returns the details of a given component. Implementations are expected to be thread safe.
|
||||
*/
|
||||
public interface ComponentDetailsStorage {
|
||||
|
||||
/**
|
||||
* Returns with the details of a given component if known. A component is know if it was present in the last call
|
||||
* of {@code #addComponentDetails}.
|
||||
*
|
||||
* @param componentId The component's identifier.
|
||||
*
|
||||
* @return A map of the details used for presenting status history if the component is known. Empty map otherwise.
|
||||
*/
|
||||
Map<String, String> getDetails(String componentId);
|
||||
|
||||
/**
|
||||
* Sets the component details for the storage. The call overwrites the previous values.
|
||||
*
|
||||
* @param componentDetails The known component details.
|
||||
*/
|
||||
void setComponentDetails(Map<String, ComponentDetails> componentDetails);
|
||||
}
|
|
@ -0,0 +1,123 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import io.questdb.cairo.sql.RecordCursor;
|
||||
import io.questdb.cairo.sql.RecordCursorFactory;
|
||||
import io.questdb.griffin.SqlCompiler;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* QuestDB does not provide the possibility for deleting individual lines. Instead there is the option to drop older
|
||||
* partitions. In order to clean up older status information, the partitions are outside of the scope of data we intend
|
||||
* to keep will be deleted.
|
||||
*/
|
||||
public class EmbeddedQuestDbRolloverHandler implements Runnable {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedQuestDbRolloverHandler.class);
|
||||
|
||||
// Drop keyword is intentionally not uppercase as the query parser only recognizes it in this way
|
||||
private static final String DELETION_QUERY = "ALTER TABLE %s drop PARTITION '%s'";
|
||||
// Distinct keyword is not recognized if the date mapping is not within an inner query
|
||||
static final String SELECTION_QUERY = "SELECT DISTINCT * FROM (SELECT (to_str(capturedAt, 'yyyy-MM-dd')) AS partitionName FROM %s)";
|
||||
|
||||
static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd").withZone(ZoneId.systemDefault());
|
||||
|
||||
private final QuestDbContext dbContext;
|
||||
private final List<String> tables = new ArrayList<>();
|
||||
private final int daysToKeepData;
|
||||
|
||||
public EmbeddedQuestDbRolloverHandler(final Collection<String> tables, final int daysToKeepData, final QuestDbContext dbContext) {
|
||||
this.tables.addAll(tables);
|
||||
this.dbContext = dbContext;
|
||||
this.daysToKeepData = daysToKeepData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
LOGGER.debug("Starting rollover");
|
||||
tables.forEach(tableName -> rolloverTable(tableName));
|
||||
LOGGER.debug("Finishing rollover");
|
||||
}
|
||||
|
||||
private void rolloverTable(final CharSequence tableName) {
|
||||
try {
|
||||
final Set<String> partitions = getPartitions(tableName);
|
||||
final Set<String> partitionToKeep = getPartitionsToKeep();
|
||||
|
||||
for (final String partition : partitions) {
|
||||
if (!partitionToKeep.contains(partition)) {
|
||||
deletePartition(tableName, partition);
|
||||
}
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
LOGGER.error("Could not rollover table " + tableName, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void deletePartition(final CharSequence tableName, final String partition) {
|
||||
try (final SqlCompiler compiler = dbContext.getCompiler()) {
|
||||
compiler.compile(String.format(DELETION_QUERY, new Object[]{tableName, partition}), dbContext.getSqlExecutionContext());
|
||||
} catch (final Exception e) {
|
||||
LOGGER.error("Dropping partition " + partition + " of table " + tableName + " failed", e);
|
||||
}
|
||||
}
|
||||
|
||||
private Set<String> getPartitions(final CharSequence tableName) throws Exception {
|
||||
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
|
||||
final Set<String> result = new HashSet<>();
|
||||
|
||||
try (
|
||||
final SqlCompiler compiler = dbContext.getCompiler();
|
||||
final RecordCursorFactory recordCursorFactory = compiler.compile(String.format(SELECTION_QUERY, new Object[]{tableName}), executionContext).getRecordCursorFactory();
|
||||
final RecordCursor cursor = recordCursorFactory.getCursor(executionContext);
|
||||
) {
|
||||
while (cursor.hasNext()) {
|
||||
final Record record = cursor.getRecord();
|
||||
result.add(new StringBuilder(record.getStr(0)).toString());
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private Set<String> getPartitionsToKeep() {
|
||||
final Instant now = Instant.now();
|
||||
|
||||
// Note: as only full partitions might be deleted and the status history repository works with day based partitions,
|
||||
// a partition must remain until any part of it might be the subject of request.
|
||||
final Set<String> result = new HashSet<>();
|
||||
for (int i = 0; i < daysToKeepData + 1; i++) {
|
||||
result.add(DATE_FORMATTER.format(now.minus(i, ChronoUnit.DAYS)));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,278 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import io.questdb.MessageBusImpl;
|
||||
import io.questdb.cairo.CairoConfiguration;
|
||||
import io.questdb.cairo.CairoEngine;
|
||||
import io.questdb.cairo.DefaultCairoConfiguration;
|
||||
import org.apache.commons.lang3.concurrent.BasicThreadFactory;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.apache.nifi.controller.status.ConnectionStatus;
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
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.history.questdb.QuestDbContext;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbDatabaseManager;
|
||||
import org.apache.nifi.controller.status.history.storage.BufferedWriterFlushWorker;
|
||||
import org.apache.nifi.controller.status.history.storage.BufferedWriterForStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.ComponentStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.GarbageCollectionStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.NodeStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.ProcessorStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.questdb.QuestDbConnectionStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.questdb.QuestDbGarbageCollectionStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.questdb.QuestDbNodeStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.questdb.QuestDbProcessGroupStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.questdb.QuestDbProcessorStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.questdb.QuestDbRemoteProcessGroupStatusStorage;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.nio.file.Path;
|
||||
import java.time.Instant;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class EmbeddedQuestDbStatusHistoryRepository implements StatusHistoryRepository {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedQuestDbStatusHistoryRepository.class);
|
||||
private static final int PERSIST_BATCH_SIZE = 1000;
|
||||
private static final long PERSIST_FREQUENCY = TimeUnit.MILLISECONDS.convert(5, TimeUnit.SECONDS);
|
||||
private static final long ROLL_FREQUENCY = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
|
||||
|
||||
|
||||
private final InMemoryComponentDetailsStorage componentDetailsProvider = new InMemoryComponentDetailsStorage();
|
||||
private final ScheduledExecutorService scheduledExecutorService = Executors
|
||||
.newScheduledThreadPool(3, new BasicThreadFactory.Builder().namingPattern("EmbeddedQuestDbStatusHistoryRepositoryWorker-%d").build());
|
||||
|
||||
private final QuestDbContext dbContext;
|
||||
private final long persistFrequency;
|
||||
private final int daysToKeepNodeData;
|
||||
private final int daysToKeepComponentData;
|
||||
|
||||
private final ProcessorStatusStorage processorStatusStorage;
|
||||
private final ComponentStatusStorage<ConnectionStatus> connectionStatusStorage;
|
||||
private final ComponentStatusStorage<ProcessGroupStatus> processGroupStatusStorage;
|
||||
private final ComponentStatusStorage<RemoteProcessGroupStatus> remoteProcessGroupStatusStorage;
|
||||
private final NodeStatusStorage nodeStatusStorage;
|
||||
private final GarbageCollectionStatusStorage garbageCollectionStatusStorage;
|
||||
|
||||
private final BufferedWriterForStatusStorage<ProcessorStatus> processorStatusWriter;
|
||||
private final BufferedWriterForStatusStorage<ConnectionStatus> connectionStatusWriter;
|
||||
private final BufferedWriterForStatusStorage<ProcessGroupStatus> processGroupStatusWriter;
|
||||
private final BufferedWriterForStatusStorage<RemoteProcessGroupStatus> remoteProcessGroupStatusWriter;
|
||||
private final BufferedWriterForStatusStorage<NodeStatus> nodeStatusWriter;
|
||||
private final BufferedWriterForStatusStorage<GarbageCollectionStatus> garbageCollectionStatusWriter;
|
||||
|
||||
/**
|
||||
* Default no args constructor for service loading only
|
||||
*/
|
||||
public EmbeddedQuestDbStatusHistoryRepository() {
|
||||
dbContext = null;
|
||||
persistFrequency = PERSIST_FREQUENCY;
|
||||
daysToKeepNodeData = -1;
|
||||
daysToKeepComponentData = -1;
|
||||
|
||||
processorStatusStorage = null;
|
||||
connectionStatusStorage = null;
|
||||
processGroupStatusStorage = null;
|
||||
remoteProcessGroupStatusStorage = null;
|
||||
nodeStatusStorage = null;
|
||||
garbageCollectionStatusStorage = null;
|
||||
|
||||
processorStatusWriter = null;
|
||||
connectionStatusWriter = null;
|
||||
processGroupStatusWriter = null;
|
||||
remoteProcessGroupStatusWriter = null;
|
||||
nodeStatusWriter = null;
|
||||
garbageCollectionStatusWriter = null;
|
||||
}
|
||||
|
||||
public EmbeddedQuestDbStatusHistoryRepository(final NiFiProperties niFiProperties) {
|
||||
this(niFiProperties, PERSIST_FREQUENCY);
|
||||
}
|
||||
|
||||
EmbeddedQuestDbStatusHistoryRepository(final NiFiProperties niFiProperties, final long persistFrequency) {
|
||||
final Path persistLocation = niFiProperties.getQuestDbStatusRepositoryPath();
|
||||
final CairoConfiguration configuration = new DefaultCairoConfiguration(persistLocation.toString());
|
||||
QuestDbDatabaseManager.checkDatabaseStatus(persistLocation);
|
||||
|
||||
this.persistFrequency = persistFrequency;
|
||||
daysToKeepNodeData = getDaysToKeepNodeData(niFiProperties);
|
||||
daysToKeepComponentData = getDaysToKeepComponentData(niFiProperties);
|
||||
dbContext = new QuestDbContext(new CairoEngine(configuration), new MessageBusImpl());
|
||||
|
||||
nodeStatusStorage = new QuestDbNodeStatusStorage(dbContext);
|
||||
garbageCollectionStatusStorage = new QuestDbGarbageCollectionStatusStorage(dbContext);
|
||||
processorStatusStorage = new QuestDbProcessorStatusStorage(dbContext, componentDetailsProvider);
|
||||
connectionStatusStorage = new QuestDbConnectionStatusStorage(dbContext, componentDetailsProvider);
|
||||
processGroupStatusStorage = new QuestDbProcessGroupStatusStorage(dbContext, componentDetailsProvider);
|
||||
remoteProcessGroupStatusStorage = new QuestDbRemoteProcessGroupStatusStorage(dbContext, componentDetailsProvider);
|
||||
|
||||
nodeStatusWriter = new BufferedWriterForStatusStorage<>(nodeStatusStorage, PERSIST_BATCH_SIZE);
|
||||
garbageCollectionStatusWriter = new BufferedWriterForStatusStorage<>(garbageCollectionStatusStorage, PERSIST_BATCH_SIZE);
|
||||
processorStatusWriter = new BufferedWriterForStatusStorage<>(processorStatusStorage, PERSIST_BATCH_SIZE);
|
||||
connectionStatusWriter = new BufferedWriterForStatusStorage<>(connectionStatusStorage, PERSIST_BATCH_SIZE);
|
||||
processGroupStatusWriter = new BufferedWriterForStatusStorage<>(processGroupStatusStorage, PERSIST_BATCH_SIZE);
|
||||
remoteProcessGroupStatusWriter = new BufferedWriterForStatusStorage<>(remoteProcessGroupStatusStorage, PERSIST_BATCH_SIZE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
LOGGER.debug("Starting status history repository");
|
||||
|
||||
final EmbeddedQuestDbRolloverHandler nodeRolloverHandler = new EmbeddedQuestDbRolloverHandler(QuestDbDatabaseManager.getNodeTableNames(), daysToKeepNodeData, dbContext);
|
||||
final EmbeddedQuestDbRolloverHandler componentRolloverHandler = new EmbeddedQuestDbRolloverHandler(QuestDbDatabaseManager.getComponentTableNames(), daysToKeepComponentData, dbContext);
|
||||
final BufferedWriterFlushWorker writer = new BufferedWriterFlushWorker(Arrays.asList(
|
||||
nodeStatusWriter,
|
||||
garbageCollectionStatusWriter,
|
||||
processorStatusWriter,
|
||||
connectionStatusWriter,
|
||||
processGroupStatusWriter,
|
||||
remoteProcessGroupStatusWriter
|
||||
));
|
||||
|
||||
scheduledExecutorService.scheduleWithFixedDelay(nodeRolloverHandler, 0, ROLL_FREQUENCY, TimeUnit.MILLISECONDS);
|
||||
scheduledExecutorService.scheduleWithFixedDelay(componentRolloverHandler, 0, ROLL_FREQUENCY, TimeUnit.MILLISECONDS);
|
||||
scheduledExecutorService.scheduleWithFixedDelay(writer, 0, persistFrequency, TimeUnit.MILLISECONDS);
|
||||
|
||||
LOGGER.debug("Status history repository is started");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
LOGGER.debug("Status history repository started to shut down");
|
||||
scheduledExecutorService.shutdown();
|
||||
dbContext.close();
|
||||
LOGGER.debug("Status history repository has been shut down");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void capture(
|
||||
final NodeStatus nodeStatus,
|
||||
final ProcessGroupStatus rootGroupStatus,
|
||||
final List<GarbageCollectionStatus> garbageCollectionStatus,
|
||||
final Date capturedAt
|
||||
) {
|
||||
captureNodeLevelStatus(nodeStatus, garbageCollectionStatus, capturedAt.toInstant());
|
||||
captureComponentLevelStatus(rootGroupStatus, capturedAt.toInstant());
|
||||
}
|
||||
|
||||
private void captureComponentLevelStatus(final ProcessGroupStatus rootGroupStatus, final Instant capturedAt) {
|
||||
captureComponents(rootGroupStatus, capturedAt);
|
||||
updateComponentDetails(rootGroupStatus);
|
||||
}
|
||||
|
||||
private void captureNodeLevelStatus(final NodeStatus nodeStatus, final List<GarbageCollectionStatus> garbageCollectionStatus, final Instant capturedAt) {
|
||||
nodeStatusWriter.collect(new ImmutablePair<>(capturedAt, nodeStatus));
|
||||
garbageCollectionStatus.forEach(s -> garbageCollectionStatusWriter.collect(new ImmutablePair<>(capturedAt, s)));
|
||||
}
|
||||
|
||||
private void captureComponents(final ProcessGroupStatus groupStatus, final Instant capturedAt) {
|
||||
processGroupStatusWriter.collect(new ImmutablePair<>(capturedAt, groupStatus));
|
||||
groupStatus.getConnectionStatus().forEach(s -> connectionStatusWriter.collect(new ImmutablePair<>(capturedAt, s)));
|
||||
groupStatus.getRemoteProcessGroupStatus().forEach(s -> remoteProcessGroupStatusWriter.collect(new ImmutablePair<>(capturedAt, s)));
|
||||
groupStatus.getProcessorStatus().forEach(s -> processorStatusWriter.collect(new ImmutablePair<>(capturedAt, s)));
|
||||
groupStatus.getProcessGroupStatus().forEach(childGroupStatus -> captureComponents(childGroupStatus, capturedAt));
|
||||
}
|
||||
|
||||
/**
|
||||
* Before the first capture, there will be no component detail provided!
|
||||
*
|
||||
* @param groupStatus Updates component details for components within the group, including the group itself.
|
||||
*/
|
||||
private void updateComponentDetails(final ProcessGroupStatus groupStatus) {
|
||||
// Note: details of deleted components will not be maintained (thus they are not reachable), but their status
|
||||
// information is stored in the database until rolled out.
|
||||
final Map<String, ComponentDetails> accumulator = new HashMap<>();
|
||||
updateComponentDetails(groupStatus, accumulator);
|
||||
componentDetailsProvider.setComponentDetails(accumulator);
|
||||
}
|
||||
|
||||
private void updateComponentDetails(final ProcessGroupStatus groupStatus, final Map<String, ComponentDetails> accumulator) {
|
||||
accumulator.put(groupStatus.getId(), ComponentDetails.forProcessGroup(groupStatus));
|
||||
groupStatus.getConnectionStatus().forEach(status -> accumulator.put(status.getId(), ComponentDetails.forConnection(status)));
|
||||
groupStatus.getRemoteProcessGroupStatus().forEach(status -> accumulator.put(status.getId(), ComponentDetails.forRemoteProcessGroup(status)));
|
||||
groupStatus.getProcessorStatus().forEach(status -> accumulator.put(status.getId(), ComponentDetails.forProcessor(status)));
|
||||
groupStatus.getProcessGroupStatus().forEach(childGroupStatus -> updateComponentDetails(childGroupStatus, accumulator));
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory getConnectionStatusHistory(final String connectionId, final Date start, final Date end, final int preferredDataPoints) {
|
||||
return connectionStatusStorage.read(connectionId, getStartTime(start), getEndTime(end), preferredDataPoints);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory getProcessGroupStatusHistory(final String processGroupId, final Date start, final Date end, final int preferredDataPoints) {
|
||||
return processGroupStatusStorage.read(processGroupId, getStartTime(start), getEndTime(end), preferredDataPoints);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory getProcessorStatusHistory(final String processorId, final Date start, final Date end, final int preferredDataPoints, final boolean includeCounters) {
|
||||
return includeCounters
|
||||
? processorStatusStorage.readWithCounter(processorId, getStartTime(start), getEndTime(end), preferredDataPoints)
|
||||
: processorStatusStorage.read(processorId, getStartTime(start), getEndTime(end), preferredDataPoints);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroupId, final Date start, final Date end, final int preferredDataPoints) {
|
||||
return remoteProcessGroupStatusStorage.read(remoteGroupId, getStartTime(start), getEndTime(end), preferredDataPoints);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GarbageCollectionHistory getGarbageCollectionHistory(final Date start, final Date end) {
|
||||
return garbageCollectionStatusStorage.read(getStartTime(start), getEndTime(end));
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory getNodeStatusHistory(final Date start, final Date end) {
|
||||
return nodeStatusStorage.read(getStartTime(start), getEndTime(end));
|
||||
}
|
||||
|
||||
private Integer getDaysToKeepNodeData(final NiFiProperties niFiProperties) {
|
||||
return niFiProperties.getIntegerProperty(
|
||||
NiFiProperties.STATUS_REPOSITORY_QUESTDB_PERSIST_NODE_DAYS,
|
||||
NiFiProperties.DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_NODE_DAYS);
|
||||
}
|
||||
|
||||
private Integer getDaysToKeepComponentData(final NiFiProperties niFiProperties) {
|
||||
return niFiProperties.getIntegerProperty(
|
||||
NiFiProperties.STATUS_REPOSITORY_QUESTDB_PERSIST_COMPONENT_DAYS,
|
||||
NiFiProperties.DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_COMPONENT_DAYS);
|
||||
}
|
||||
|
||||
private Instant getStartTime(final Date start) {
|
||||
if (start == null) {
|
||||
return Instant.now().minus(1, ChronoUnit.DAYS);
|
||||
} else {
|
||||
return start.toInstant();
|
||||
}
|
||||
}
|
||||
|
||||
private Instant getEndTime(final Date end) {
|
||||
return (end == null) ? Instant.now() : end.toInstant();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class EmptyStatusHistory implements StatusHistory {
|
||||
private final Date dateGenerated = new Date();
|
||||
|
||||
@Override
|
||||
public Date getDateGenerated() {
|
||||
return dateGenerated;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getComponentDetails() {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StatusSnapshot> getStatusSnapshots() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* Implementation of {@link ComponentDetailsStorage} using in memory data structure.
|
||||
*/
|
||||
public class InMemoryComponentDetailsStorage implements ComponentDetailsStorage {
|
||||
private final AtomicReference<Map<String, ComponentDetails>> componentDetails = new AtomicReference<>(new HashMap<>());
|
||||
|
||||
@Override
|
||||
public Map<String, String> getDetails(final String componentId) {
|
||||
final ComponentDetails componentDetails = this.componentDetails.get().get(componentId);
|
||||
return componentDetails == null ? Collections.emptyMap() : componentDetails.toMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setComponentDetails(final Map<String, ComponentDetails> componentDetails) {
|
||||
this.componentDetails.set(componentDetails);
|
||||
}
|
||||
}
|
|
@ -111,8 +111,11 @@ public enum ProcessGroupStatusDescriptor {
|
|||
return descriptor;
|
||||
}
|
||||
|
||||
|
||||
private static long calculateTaskMillis(final ProcessGroupStatus status) {
|
||||
return TimeUnit.MILLISECONDS.convert(calculateTaskNanos(status), TimeUnit.NANOSECONDS);
|
||||
}
|
||||
|
||||
private static long calculateTaskNanos(final ProcessGroupStatus status) {
|
||||
long nanos = 0L;
|
||||
|
||||
for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
|
||||
|
@ -120,9 +123,9 @@ public enum ProcessGroupStatusDescriptor {
|
|||
}
|
||||
|
||||
for (final ProcessGroupStatus childStatus : status.getProcessGroupStatus()) {
|
||||
nanos += calculateTaskMillis(childStatus);
|
||||
nanos += calculateTaskNanos(childStatus);
|
||||
}
|
||||
|
||||
return TimeUnit.MILLISECONDS.convert(nanos, TimeUnit.NANOSECONDS);
|
||||
return nanos;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
@ -39,7 +38,8 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class VolatileComponentStatusRepository implements ComponentStatusRepository {
|
||||
public class VolatileComponentStatusRepository implements StatusHistoryRepository {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(VolatileComponentStatusRepository.class);
|
||||
|
||||
private static final Set<MetricDescriptor<?>> DEFAULT_PROCESSOR_METRICS = Arrays.stream(ProcessorStatusDescriptor.values())
|
||||
|
@ -95,11 +95,6 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
|
|||
nodeStatuses = new RingBuffer<>(numDataPoints);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void capture(final NodeStatus nodeStatus, final ProcessGroupStatus rootGroupStatus, final List<GarbageCollectionStatus> gcStatus) {
|
||||
capture(nodeStatus, rootGroupStatus, gcStatus, new Date());
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void capture(final NodeStatus nodeStatus, final ProcessGroupStatus rootGroupStatus, final List<GarbageCollectionStatus> gcStatus, final Date timestamp) {
|
||||
final Date evicted = timestamps.add(timestamp);
|
||||
|
@ -156,12 +151,6 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
|
|||
procHistory.update(statusSnapshot, componentDetails);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Date getLastCaptureDate() {
|
||||
return new Date(lastCaptureTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory getProcessorStatusHistory(final String processorId, final Date start, final Date end, final int preferredDataPoints, final boolean includeCounters) {
|
||||
return getStatusHistory(processorId, includeCounters, DEFAULT_PROCESSOR_METRICS, start, end, preferredDataPoints);
|
||||
|
@ -183,7 +172,7 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
|
|||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory getNodeStatusHistory() {
|
||||
public StatusHistory getNodeStatusHistory(final Date start, final Date end) {
|
||||
final List<NodeStatus> nodeStatusList = nodeStatuses.asList();
|
||||
final List<List<GarbageCollectionStatus>> gcStatusList = gcStatuses.asList();
|
||||
final LinkedList<StatusSnapshot> snapshots = new LinkedList<>();
|
||||
|
@ -358,7 +347,7 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
|
|||
final Date start, final Date end, final int preferredDataPoints) {
|
||||
final ComponentStatusHistory history = componentStatusHistories.get(componentId);
|
||||
if (history == null) {
|
||||
return createEmptyStatusHistory();
|
||||
return new EmptyStatusHistory();
|
||||
}
|
||||
final List<Date> dates = filterDates(start, end, preferredDataPoints);
|
||||
return history.toStatusHistory(dates, includeCounters, defaultMetricDescriptors);
|
||||
|
@ -381,28 +370,6 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
|
|||
}
|
||||
|
||||
|
||||
private StatusHistory createEmptyStatusHistory() {
|
||||
final Date dateGenerated = new Date();
|
||||
|
||||
return new StatusHistory() {
|
||||
@Override
|
||||
public Date getDateGenerated() {
|
||||
return dateGenerated;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getComponentDetails() {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StatusSnapshot> getStatusSnapshots() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public GarbageCollectionHistory getGarbageCollectionHistory(final Date start, final Date end) {
|
||||
final StandardGarbageCollectionHistory history = new StandardGarbageCollectionHistory();
|
||||
|
@ -424,4 +391,14 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
|
|||
|
||||
return history;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
// Nothing to do
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
// Nothing to do
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.questdb;
|
||||
|
||||
import io.questdb.MessageBus;
|
||||
import io.questdb.cairo.CairoConfiguration;
|
||||
import io.questdb.cairo.CairoEngine;
|
||||
import io.questdb.griffin.SqlCompiler;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import io.questdb.griffin.SqlExecutionContextImpl;
|
||||
|
||||
public class QuestDbContext {
|
||||
private final CairoEngine engine;
|
||||
private final MessageBus messageBus;
|
||||
|
||||
public QuestDbContext(final CairoEngine engine, final MessageBus messageBus) {
|
||||
this.engine = engine;
|
||||
this.messageBus = messageBus;
|
||||
}
|
||||
|
||||
public CairoEngine getEngine() {
|
||||
return engine;
|
||||
}
|
||||
|
||||
public CairoConfiguration getConfiguration() {
|
||||
return engine.getConfiguration();
|
||||
}
|
||||
|
||||
public SqlExecutionContext getSqlExecutionContext() {
|
||||
return new SqlExecutionContextImpl(engine.getConfiguration(), messageBus, 1);
|
||||
}
|
||||
|
||||
public SqlCompiler getCompiler() {
|
||||
return new SqlCompiler(engine, messageBus);
|
||||
}
|
||||
|
||||
public void close() {
|
||||
engine.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,180 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.questdb;
|
||||
|
||||
import io.questdb.MessageBusImpl;
|
||||
import io.questdb.cairo.CairoConfiguration;
|
||||
import io.questdb.cairo.CairoEngine;
|
||||
import io.questdb.cairo.DefaultCairoConfiguration;
|
||||
import io.questdb.griffin.SqlCompiler;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import io.questdb.griffin.SqlExecutionContextImpl;
|
||||
import org.apache.nifi.util.FileUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* The database manager is responsible for checking and maintaining the health of the database during startup.
|
||||
*/
|
||||
public final class QuestDbDatabaseManager {
|
||||
private enum DatabaseStatus {
|
||||
HEALTHY, NON_EXISTING, CORRUPTED;
|
||||
}
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(QuestDbDatabaseManager.class);
|
||||
private static final Set<String> COMPONENT_TABLES = new HashSet<>();
|
||||
private static final Set<String> NODE_TABLES = new HashSet<>();
|
||||
|
||||
static {
|
||||
COMPONENT_TABLES.add("componentCounter");
|
||||
COMPONENT_TABLES.add("connectionStatus");
|
||||
COMPONENT_TABLES.add("processGroupStatus");
|
||||
COMPONENT_TABLES.add("remoteProcessGroupStatus");
|
||||
COMPONENT_TABLES.add("processorStatus");
|
||||
|
||||
NODE_TABLES.add("nodeStatus");
|
||||
NODE_TABLES.add("garbageCollectionStatus");
|
||||
NODE_TABLES.add("storageStatus");
|
||||
}
|
||||
|
||||
private QuestDbDatabaseManager() {
|
||||
// Should not be instantiated.
|
||||
}
|
||||
|
||||
public static void checkDatabaseStatus(final Path persistLocation) {
|
||||
final QuestDbDatabaseManager.DatabaseStatus databaseStatus = getDatabaseStatus(persistLocation);
|
||||
LOGGER.debug("Starting status repository. It's estimated status is {}", databaseStatus);
|
||||
|
||||
if (databaseStatus == QuestDbDatabaseManager.DatabaseStatus.NON_EXISTING) {
|
||||
createDatabase(persistLocation);
|
||||
} else if (databaseStatus == QuestDbDatabaseManager.DatabaseStatus.CORRUPTED) {
|
||||
throw new RuntimeException("The database is corrupted. The expected set of tables is not matching with the reachable tables.");
|
||||
}
|
||||
}
|
||||
|
||||
private static DatabaseStatus getDatabaseStatus(final Path persistLocation) {
|
||||
|
||||
if (!checkPersistentLocationExists(persistLocation)) {
|
||||
return DatabaseStatus.NON_EXISTING;
|
||||
}
|
||||
|
||||
if (checkPersistentLocationExists(persistLocation) && checkPersistentLocationIsEmpty(persistLocation)) {
|
||||
return DatabaseStatus.NON_EXISTING;
|
||||
}
|
||||
|
||||
if (!checkTablesAreInPlace(persistLocation) || !checkConnection(persistLocation)) {
|
||||
return DatabaseStatus.CORRUPTED;
|
||||
}
|
||||
|
||||
return DatabaseStatus.HEALTHY;
|
||||
}
|
||||
|
||||
private static boolean checkPersistentLocationExists(final Path persistLocation) {
|
||||
final File persistLocationDirectory = persistLocation.toFile();
|
||||
return persistLocationDirectory.exists() && persistLocationDirectory.isDirectory();
|
||||
}
|
||||
|
||||
private static boolean checkPersistentLocationIsEmpty(final Path persistLocation) {
|
||||
final File persistLocationDirectory = persistLocation.toFile();
|
||||
return persistLocationDirectory.list().length == 0;
|
||||
}
|
||||
|
||||
private static boolean checkTablesAreInPlace(final Path persistLocation) {
|
||||
final File persistLocationDirectory = persistLocation.toFile();
|
||||
final Map<String, File> databaseFiles = Arrays.stream(persistLocationDirectory.listFiles())
|
||||
.collect(Collectors.toMap(f -> f.getAbsolutePath().substring(persistLocationDirectory.getAbsolutePath().length() + 1), f -> f));
|
||||
|
||||
final Set<String> expectedTables = new HashSet<>();
|
||||
expectedTables.addAll(NODE_TABLES);
|
||||
expectedTables.addAll(COMPONENT_TABLES);
|
||||
|
||||
for (final String expectedTable : expectedTables) {
|
||||
if (!databaseFiles.containsKey(expectedTable) || !databaseFiles.get(expectedTable).isDirectory()) {
|
||||
LOGGER.error("Missing table during database status check: ", expectedTable);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private static boolean checkConnection(final Path persistLocation) {
|
||||
final CairoConfiguration configuration = new DefaultCairoConfiguration(persistLocation.toFile().getAbsolutePath());
|
||||
|
||||
try (
|
||||
final CairoEngine engine = new CairoEngine(configuration);
|
||||
) {
|
||||
LOGGER.info("Connection to database was successful");
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
LOGGER.error("Error during connection to database", e);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private static void createDatabase(final Path persistLocation) {
|
||||
LOGGER.info("Creating database");
|
||||
final CairoConfiguration configuration;
|
||||
|
||||
try {
|
||||
FileUtils.ensureDirectoryExistAndCanReadAndWrite(persistLocation.toFile());
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException("Could not create database folder " + persistLocation.toAbsolutePath().toString(), e);
|
||||
}
|
||||
|
||||
configuration = new DefaultCairoConfiguration(persistLocation.toFile().getAbsolutePath());
|
||||
|
||||
try (
|
||||
final CairoEngine engine = new CairoEngine(configuration);
|
||||
final SqlCompiler compiler = new SqlCompiler(engine);
|
||||
) {
|
||||
final SqlExecutionContext context = new SqlExecutionContextImpl(engine.getConfiguration(), new MessageBusImpl(), 1);
|
||||
|
||||
// Node status tables
|
||||
compiler.compile(QuestDbQueries.CREATE_GARBAGE_COLLECTION_STATUS, context);
|
||||
compiler.compile(QuestDbQueries.CREATE_NODE_STATUS, context);
|
||||
compiler.compile(QuestDbQueries.CREATE_STORAGE_STATUS, context);
|
||||
|
||||
// Component status tables
|
||||
compiler.compile(QuestDbQueries.CREATE_CONNECTION_STATUS, context);
|
||||
compiler.compile(QuestDbQueries.CREATE_PROCESS_GROUP_STATUS, context);
|
||||
compiler.compile(QuestDbQueries.CREATE_REMOTE_PROCESS_GROUP_STATUS, context);
|
||||
compiler.compile(QuestDbQueries.CREATE_PROCESSOR_STATUS, context);
|
||||
compiler.compile(QuestDbQueries.CREATE_COMPONENT_COUNTER, context);
|
||||
|
||||
LOGGER.info("Database is created");
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException("Could not create database!", e);
|
||||
}
|
||||
}
|
||||
|
||||
public static Set<String> getNodeTableNames() {
|
||||
return NODE_TABLES;
|
||||
}
|
||||
|
||||
public static Set<String> getComponentTableNames() {
|
||||
return COMPONENT_TABLES;
|
||||
}
|
||||
}
|
|
@ -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.history.questdb;
|
||||
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import io.questdb.cairo.sql.RecordCursor;
|
||||
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Template for reading list of entities from database.
|
||||
*
|
||||
* @param <E> The entity type represented by a single database record.
|
||||
* @param <R> The result of the selection. Might be an aggregated value or collection.
|
||||
*/
|
||||
public class QuestDbEntityReadingTemplate<E, R> extends QuestDbReadingTemplate<R> {
|
||||
private final Function<Record, E> mapper;
|
||||
private final Function<List<E>, R> aggregator;
|
||||
|
||||
/**
|
||||
* @param query The query to execute. Parameters might be added using the format specified by {@link String#format}.
|
||||
* @param mapper Responsible for mapping one database record into one entity object.
|
||||
* @param aggregator Might process the list of selected entities after the query has been executed.
|
||||
* @param errorResult Error handler in case of an exception arises during the execution.
|
||||
*/
|
||||
public QuestDbEntityReadingTemplate(
|
||||
final String query,
|
||||
final Function<Record, E> mapper,
|
||||
final Function<List<E>, R> aggregator,
|
||||
final Function<Exception, R> errorResult) {
|
||||
super(query, errorResult);
|
||||
this.mapper = mapper;
|
||||
this.aggregator = aggregator;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected R processResult(final RecordCursor cursor) {
|
||||
final List<E> entities = new LinkedList<>();
|
||||
|
||||
while (cursor.hasNext()) {
|
||||
entities.add(mapper.apply(cursor.getRecord()));
|
||||
}
|
||||
|
||||
return aggregator.apply(entities);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.questdb;
|
||||
|
||||
import io.questdb.cairo.TableWriter;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BiConsumer;
|
||||
|
||||
/**
|
||||
* Writes entry to the database with the given measurement time.
|
||||
*
|
||||
* @param <E> Entry type.
|
||||
*/
|
||||
public class QuestDbEntityWritingTemplate<E> extends QuestDbWritingTemplate<Pair<Instant, E>> {
|
||||
private final BiConsumer<E, TableWriter.Row> fillRow;
|
||||
|
||||
/**
|
||||
* @param tableName Name of the target table.
|
||||
* @param fillRow Responsible for filling a row based on the entry.
|
||||
*/
|
||||
public QuestDbEntityWritingTemplate(final String tableName, final BiConsumer<E, TableWriter.Row> fillRow) {
|
||||
super(tableName);
|
||||
this.fillRow = fillRow;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void addRows(final TableWriter tableWriter, final Collection<Pair<Instant, E>> entries) {
|
||||
entries.forEach(statusEntry -> {
|
||||
final long capturedAt = TimeUnit.MILLISECONDS.toMicros(statusEntry.getLeft().toEpochMilli());
|
||||
final TableWriter.Row row = tableWriter.newRow(capturedAt);
|
||||
fillRow.accept(statusEntry.getRight(), row);
|
||||
row.append();
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,129 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.questdb;
|
||||
|
||||
final class QuestDbQueries {
|
||||
|
||||
private QuestDbQueries() {
|
||||
// Should not be instantiated!
|
||||
}
|
||||
|
||||
// Node status tables
|
||||
|
||||
public static final String CREATE_NODE_STATUS =
|
||||
"CREATE TABLE nodeStatus (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"freeHeap LONG," +
|
||||
"usedHeap LONG," +
|
||||
"heapUtilization LONG," +
|
||||
"freeNonHeap LONG," +
|
||||
"usedNonHeap LONG," +
|
||||
"openFileHandlers LONG," +
|
||||
"processorLoadAverage DOUBLE," +
|
||||
"totalThreads LONG," +
|
||||
"eventDrivenThreads LONG," +
|
||||
"timerDrivenThreads LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
public static final String CREATE_STORAGE_STATUS =
|
||||
"CREATE TABLE storageStatus (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"name SYMBOL capacity 256 nocache," +
|
||||
"storageType SHORT," +
|
||||
"freeSpace LONG," +
|
||||
"usedSpace LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
public static final String CREATE_GARBAGE_COLLECTION_STATUS =
|
||||
"CREATE TABLE garbageCollectionStatus (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"memoryManagerName SYMBOL capacity 4 nocache," +
|
||||
"collectionCount LONG," +
|
||||
"collectionMinis LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
// Component status tables
|
||||
|
||||
public static final String CREATE_PROCESSOR_STATUS =
|
||||
"CREATE TABLE processorStatus (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"componentId SYMBOL capacity 2000 nocache index capacity 1500," +
|
||||
"bytesRead LONG," +
|
||||
"bytesWritten LONG," +
|
||||
"bytesTransferred LONG," +
|
||||
"inputBytes LONG," +
|
||||
"inputCount LONG," +
|
||||
"outputBytes LONG," +
|
||||
"outputCount LONG," +
|
||||
"taskCount LONG," +
|
||||
"taskMillis LONG," +
|
||||
"taskNanos LONG," +
|
||||
"flowFilesRemoved LONG," +
|
||||
"averageLineageDuration LONG," +
|
||||
"averageTaskNanos LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
public static final String CREATE_CONNECTION_STATUS =
|
||||
"CREATE TABLE connectionStatus (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"componentId SYMBOL capacity 2000 nocache index capacity 1500," +
|
||||
"inputBytes LONG," +
|
||||
"inputCount LONG," +
|
||||
"outputBytes LONG," +
|
||||
"outputCount LONG," +
|
||||
"queuedBytes LONG," +
|
||||
"queuedCount LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
public static final String CREATE_PROCESS_GROUP_STATUS =
|
||||
"CREATE TABLE processGroupStatus (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"componentId SYMBOL capacity 2000 nocache index capacity 1500," +
|
||||
"bytesRead LONG," +
|
||||
"bytesWritten LONG," +
|
||||
"bytesTransferred LONG," +
|
||||
"inputBytes LONG," +
|
||||
"inputCount LONG," +
|
||||
"outputBytes LONG," +
|
||||
"outputCount LONG," +
|
||||
"queuedBytes LONG," +
|
||||
"queuedCount LONG," +
|
||||
"taskMillis LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
public static final String CREATE_REMOTE_PROCESS_GROUP_STATUS =
|
||||
"CREATE TABLE remoteProcessGroupStatus (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"componentId SYMBOL capacity 2000 nocache index capacity 1500," +
|
||||
"sentBytes LONG," +
|
||||
"sentCount LONG," +
|
||||
"receivedBytes LONG," +
|
||||
"receivedCount LONG," +
|
||||
"receivedBytesPerSecond LONG," +
|
||||
"sentBytesPerSecond LONG," +
|
||||
"totalBytesPerSecond LONG," +
|
||||
"averageLineageDuration LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
public static final String CREATE_COMPONENT_COUNTER =
|
||||
"CREATE TABLE componentCounter (" +
|
||||
"capturedAt TIMESTAMP," +
|
||||
"componentId SYMBOL capacity 2000 nocache index capacity 1500," +
|
||||
"name SYMBOL capacity 256 nocache," +
|
||||
"value LONG" +
|
||||
") TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.questdb;
|
||||
|
||||
import io.questdb.cairo.CairoEngine;
|
||||
import io.questdb.cairo.sql.RecordCursor;
|
||||
import io.questdb.cairo.sql.RecordCursorFactory;
|
||||
import io.questdb.griffin.SqlCompiler;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Template class for executing selection from QuestDB. The {@code readEntities} might be called multiple times, it will
|
||||
* execute the query multiple times independently, always with the given parameters.
|
||||
*
|
||||
* @param <R> The result of the selection. Might be an aggregated value or collection.
|
||||
*/
|
||||
abstract public class QuestDbReadingTemplate<R> {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(QuestDbReadingTemplate.class);
|
||||
|
||||
private final String query;
|
||||
private final Function<Exception, R> errorResult;
|
||||
|
||||
/**
|
||||
* @param query The query to execute. Parameters might be added using the format specified by {@link String#format}.
|
||||
* @param errorResult Error handler in case of an exception arises during the execution.
|
||||
*/
|
||||
public QuestDbReadingTemplate(
|
||||
final String query,
|
||||
final Function<Exception, R> errorResult) {
|
||||
this.query = query;
|
||||
this.errorResult = errorResult;
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes the query and returns with the result.
|
||||
*
|
||||
* @param engine The database engine.
|
||||
* @param context The execution context.
|
||||
* @param parameters Parameters (is any) in the order of appearance in the query string.
|
||||
*
|
||||
* @return End result of the query, after possible procession by {@link #processResult(RecordCursor)}
|
||||
*/
|
||||
public R read(final CairoEngine engine, final SqlExecutionContext context, final List<Object> parameters) {
|
||||
try (
|
||||
final SqlCompiler compiler = new SqlCompiler(engine);
|
||||
final RecordCursorFactory factory = compiler.compile(String.format(query, parameters.toArray()), context).getRecordCursorFactory();
|
||||
final RecordCursor cursor = factory.getCursor(context);
|
||||
) {
|
||||
return processResult(cursor);
|
||||
} catch (final Exception e) {
|
||||
LOGGER.error("Error during reading from database", e);
|
||||
return errorResult.apply(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract R processResult(RecordCursor cursor);
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.questdb;
|
||||
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class QuestDbStatusSnapshotMapper<T> implements Function<Record, StandardStatusSnapshot> {
|
||||
private final Map<Integer, MetricDescriptor<T>> metrics = new HashMap<>();
|
||||
|
||||
public QuestDbStatusSnapshotMapper(final Map<Integer, MetricDescriptor<T>> metrics) {
|
||||
this.metrics.putAll(metrics);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StandardStatusSnapshot apply(final Record record) {
|
||||
final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(new HashSet<>(metrics.values()));
|
||||
snapshot.setTimestamp(new Date(TimeUnit.MICROSECONDS.toMillis(record.getTimestamp(0))));
|
||||
metrics.keySet().forEach(ordinal -> snapshot.addStatusMetric(metrics.get(ordinal), record.getLong(ordinal)));
|
||||
return snapshot;
|
||||
}
|
||||
}
|
|
@ -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.controller.status.history.questdb;
|
||||
|
||||
import io.questdb.cairo.CairoEngine;
|
||||
import io.questdb.cairo.TableWriter;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* Template for writing entries into QuestDb.
|
||||
*
|
||||
* @param <T> The type of the entry.
|
||||
*/
|
||||
public abstract class QuestDbWritingTemplate<T> {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(QuestDbWritingTemplate.class);
|
||||
|
||||
private final String tableName;
|
||||
|
||||
/**
|
||||
* @param tableName Name of the target table.
|
||||
*/
|
||||
protected QuestDbWritingTemplate(final String tableName) {
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts the entries into the database.
|
||||
*
|
||||
* @param engine QuestDB engine.
|
||||
* @param context Execution context.
|
||||
* @param entries Entries to insert.
|
||||
*/
|
||||
public void insert(final CairoEngine engine, final SqlExecutionContext context, final Collection<T> entries) {
|
||||
if (entries.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
try (
|
||||
final TableWriter tableWriter = engine.getWriter(context.getCairoSecurityContext(), tableName);
|
||||
) {
|
||||
addRows(tableWriter, entries);
|
||||
tableWriter.commit();
|
||||
} catch (final Exception e) {
|
||||
LOGGER.error("Error happened during writing into table " + tableName, e);
|
||||
} finally {
|
||||
engine.releaseInactive();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Populating {@link TableWriter} with data extracted from entries.
|
||||
*
|
||||
* @param tableWriter Table writer.
|
||||
* @param entries List of entries.
|
||||
*/
|
||||
abstract protected void addRows(TableWriter tableWriter, Collection<T> entries);
|
||||
}
|
|
@ -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.history.storage;
|
||||
|
||||
/**
|
||||
* Wraps a writer object in order to buffer incoming store requests and dispatch the incoming store requests in batches.
|
||||
*/
|
||||
public interface BufferedEntryWriter<T> {
|
||||
|
||||
/**
|
||||
* Collects an entity to write into the internal buffer.
|
||||
*
|
||||
* @param entryToStore The entry to store.
|
||||
*/
|
||||
void collect(T entryToStore);
|
||||
|
||||
/**
|
||||
* Initiates the store of the payload by sending buffered items to store.
|
||||
*/
|
||||
void flush();
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class BufferedWriterFlushWorker implements Runnable {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(BufferedWriterFlushWorker.class);
|
||||
|
||||
private final List<BufferedEntryWriter<?>> bufferedWriterList = new ArrayList<>();
|
||||
|
||||
public BufferedWriterFlushWorker(final List<BufferedEntryWriter<?>> bufferedWriterList) {
|
||||
this.bufferedWriterList.addAll(bufferedWriterList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
bufferedWriterList.forEach(bufferedWriter -> bufferedWriter.flush());
|
||||
} catch (final Exception e) {
|
||||
LOGGER.error("Error happened during calling flush.", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
public class BufferedWriterForStatusStorage<T> implements BufferedEntryWriter<Pair<Instant, T>> {
|
||||
private final BlockingQueue<Pair<Instant, T>> queue = new LinkedBlockingQueue<>();
|
||||
private final StatusStorage<T> statusStorage;
|
||||
private final int batchSize;
|
||||
|
||||
public BufferedWriterForStatusStorage(final StatusStorage<T> statusStorage, final int batchSize) {
|
||||
this.statusStorage = statusStorage;
|
||||
this.batchSize = batchSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(final Pair<Instant, T> entryToStore) {
|
||||
queue.add(entryToStore);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush() {
|
||||
final ArrayList<Pair<Instant, T>> entries = new ArrayList<>(batchSize);
|
||||
queue.drainTo(entries, batchSize);
|
||||
|
||||
if (!entries.isEmpty()) {
|
||||
statusStorage.store(entries);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.apache.nifi.controller.status.history.StatusHistory;
|
||||
|
||||
import java.time.Instant;
|
||||
|
||||
/**
|
||||
* Readable status storage for the specified component status entry type.
|
||||
*
|
||||
* @param <T> Stored component status entry type.
|
||||
*/
|
||||
public interface ComponentStatusStorage<T> extends StatusStorage<T> {
|
||||
|
||||
/**
|
||||
* Select query template.
|
||||
*/
|
||||
String QUERY_TEMPLATE =
|
||||
"SELECT * FROM %s " +
|
||||
"WHERE componentId = '%s' " +
|
||||
"AND capturedAt > to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"AND capturedAt < to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"ORDER BY capturedAt ASC";
|
||||
|
||||
/**
|
||||
* Returns with the status history of the given component for the specified time range.
|
||||
*
|
||||
* @param componentId The component's unique id.
|
||||
* @param start Start date of the history, inclusive. In case it is not
|
||||
* specified, the history starts one day back from the current time.
|
||||
* @param end End date of the history, inclusive. In case it is not specified
|
||||
* the end date is the current time.
|
||||
* @param preferredDataPoints the preferred number of data points to return.
|
||||
* If the date range is large, the total number of data points could be far
|
||||
* too many to process. Therefore, this parameter allows the requestor to
|
||||
* indicate how many samples to return.
|
||||
*
|
||||
* @return Status history. In case the component does not exist, the result {@link StatusHistory} will be empty.
|
||||
*/
|
||||
StatusHistory read(String componentId, Instant start, Instant end, int preferredDataPoints);
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
|
||||
import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
|
||||
|
||||
import java.time.Instant;
|
||||
|
||||
/**
|
||||
* Readable status storage for garbage collection status entries.
|
||||
*/
|
||||
public interface GarbageCollectionStatusStorage extends StatusStorage<GarbageCollectionStatus> {
|
||||
|
||||
/**
|
||||
* Returns with the status history of the garbage collection for the specified time range.
|
||||
*
|
||||
* @param start Start date of the history, inclusive.
|
||||
* @param end End date of the history, inclusive.
|
||||
*
|
||||
* @return Status history.
|
||||
*/
|
||||
GarbageCollectionHistory read(Instant start, Instant end);
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.apache.nifi.controller.status.history.StatusHistory;
|
||||
|
||||
import java.time.Instant;
|
||||
|
||||
/**
|
||||
* Readable status storage for the node status entries.
|
||||
*/
|
||||
public interface NodeStatusStorage extends StatusStorage<NodeStatus> {
|
||||
|
||||
/**
|
||||
* Returns with the status history of the node for the specified time range.
|
||||
*
|
||||
* @param start Start date of the history, inclusive.
|
||||
* @param end End date of the history, inclusive.
|
||||
*
|
||||
* @return Status history.
|
||||
*/
|
||||
StatusHistory read(Instant start, Instant end);
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.apache.nifi.controller.status.ProcessorStatus;
|
||||
import org.apache.nifi.controller.status.history.StatusHistory;
|
||||
|
||||
import java.time.Instant;
|
||||
|
||||
public interface ProcessorStatusStorage extends ComponentStatusStorage<ProcessorStatus> {
|
||||
/**
|
||||
* Returns with the status history of the given component for the specified time range. Includes counters.
|
||||
*
|
||||
* @param componentId The component's unique id.
|
||||
* @param start Start date of the history, inclusive. In case it is not
|
||||
* specified, the history starts one day back from the current time.
|
||||
* @param end End date of the history, inclusive. In case it is not specified
|
||||
* the end date is the current time.
|
||||
* @param preferredDataPoints the preferred number of data points to return.
|
||||
* If the date range is large, the total number of data points could be far
|
||||
* too many to process. Therefore, this parameter allows the requestor to
|
||||
* indicate how many samples to return.
|
||||
*
|
||||
* @return Status history. In case the component does not exist, the result {@link StatusHistory} will be empty.
|
||||
*/
|
||||
StatusHistory readWithCounter(String componentId, Instant start, Instant end, int preferredDataPoints);
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents a writable status storage for a given entry type.
|
||||
*
|
||||
* @param <T> The entry type.
|
||||
*/
|
||||
public interface StatusStorage<T> {
|
||||
/**
|
||||
* Date format expected by the storage.
|
||||
*/
|
||||
String CAPTURE_DATE_FORMAT = "yyyy-MM-dd:HH:mm:ss Z";
|
||||
|
||||
/**
|
||||
* Date formatter for the database fields.
|
||||
*/
|
||||
DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern(CAPTURE_DATE_FORMAT).withZone(ZoneId.systemDefault());
|
||||
|
||||
/**
|
||||
* Stores multiple entries.
|
||||
*
|
||||
* @param statusEntries A list of pair constructs. Every pair consists of the capture time (first) and the status entry (second).
|
||||
*/
|
||||
void store(List<Pair<Instant, T>> statusEntries);
|
||||
}
|
|
@ -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.history.storage.questdb;
|
||||
|
||||
import io.questdb.cairo.TableWriter;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.nifi.controller.status.ProcessorStatus;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbWritingTemplate;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
class ComponentCounterWritingTemplate extends QuestDbWritingTemplate<Pair<Instant, ProcessorStatus>> {
|
||||
|
||||
public ComponentCounterWritingTemplate() {
|
||||
super("componentCounter");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void addRows(final TableWriter tableWriter, final Collection<Pair<Instant, ProcessorStatus>> entries) {
|
||||
for (final Pair<Instant, ProcessorStatus> entry : entries) {
|
||||
final Map<String, Long> counters = entry.getRight().getCounters();
|
||||
|
||||
if (counters != null && counters.size() > 0) {
|
||||
for (final Map.Entry<String, Long> counter : counters.entrySet()) {
|
||||
final long capturedAt = TimeUnit.MILLISECONDS.toMicros(entry.getLeft().toEpochMilli());
|
||||
final TableWriter.Row counterRow = tableWriter.newRow(capturedAt);
|
||||
counterRow.putSym(1, entry.getRight().getId());
|
||||
counterRow.putSym(2, counter.getKey());
|
||||
counterRow.putLong(3, counter.getValue());
|
||||
counterRow.append();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,111 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.nifi.controller.status.history.ComponentDetailsStorage;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.StandardStatusHistory;
|
||||
import org.apache.nifi.controller.status.history.StatusHistory;
|
||||
import org.apache.nifi.controller.status.history.StatusSnapshot;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbEntityReadingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbEntityWritingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbStatusSnapshotMapper;
|
||||
import org.apache.nifi.controller.status.history.storage.ComponentStatusStorage;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Component specific implementation of the {@link ComponentStatusStorage}.
|
||||
*
|
||||
* @param <T> Component status entry type.
|
||||
*/
|
||||
abstract class QuestDbComponentStatusStorage<T> implements ComponentStatusStorage<T> {
|
||||
|
||||
/**
|
||||
* In case of component status entries the first two columns are fixed (measurement time and component id) and all
|
||||
* the following fields are metric values using long format. The list of these are specified by the implementation class.
|
||||
*/
|
||||
private final QuestDbEntityWritingTemplate<T> writingTemplate = new QuestDbEntityWritingTemplate<>(
|
||||
getTableName(),
|
||||
(statusEntry, row) -> {
|
||||
row.putSym(1, extractId(statusEntry));
|
||||
getMetrics().keySet().forEach(ordinal -> row.putLong(ordinal, getMetrics().get(ordinal).getValueFunction().getValue(statusEntry)));
|
||||
});
|
||||
|
||||
private final Function<Record, StatusSnapshot> statusSnapshotMapper = new QuestDbStatusSnapshotMapper(getMetrics());
|
||||
|
||||
private final QuestDbEntityReadingTemplate<StatusSnapshot, List<StatusSnapshot>> readingTemplate
|
||||
= new QuestDbEntityReadingTemplate<>(QUERY_TEMPLATE, statusSnapshotMapper, e -> e, e -> Collections.emptyList());
|
||||
|
||||
private final QuestDbContext dbContext;
|
||||
private final ComponentDetailsStorage componentDetailsStorage;
|
||||
|
||||
protected QuestDbComponentStatusStorage(final QuestDbContext dbContext, final ComponentDetailsStorage componentDetailsStorage) {
|
||||
this.dbContext = dbContext;
|
||||
this.componentDetailsStorage = componentDetailsStorage;
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts unique identifier from the status entry.
|
||||
*
|
||||
* @param statusEntry The status entry.
|
||||
*
|
||||
* @return The identifier.
|
||||
*/
|
||||
abstract protected String extractId(final T statusEntry);
|
||||
|
||||
/**
|
||||
* Specifies the metrics being stored for the given kind of component.
|
||||
* .
|
||||
* @return A map of {@link MetricDescriptor} instances defines the metrics to store. The keys in the map server as column index.
|
||||
*/
|
||||
abstract protected Map<Integer, MetricDescriptor<T>> getMetrics();
|
||||
|
||||
/**
|
||||
* Returns the database table which is used to store the data.
|
||||
*
|
||||
* @return Database table name.
|
||||
*/
|
||||
abstract protected String getTableName();
|
||||
|
||||
@Override
|
||||
public StatusHistory read(final String componentId, final Instant start, final Instant end, final int preferredDataPoints) {
|
||||
final List<StatusSnapshot> snapshots = readingTemplate.read(
|
||||
dbContext.getEngine(),
|
||||
dbContext.getSqlExecutionContext(),
|
||||
Arrays.asList(getTableName(), componentId, DATE_FORMATTER.format(start), DATE_FORMATTER.format(end)));
|
||||
return new StandardStatusHistory(
|
||||
snapshots.subList(Math.max(snapshots.size() - preferredDataPoints, 0), snapshots.size()),
|
||||
componentDetailsStorage.getDetails(componentId),
|
||||
new Date()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void store(final List<Pair<Instant, T>> statusEntries) {
|
||||
writingTemplate.insert(dbContext.getEngine(), dbContext.getSqlExecutionContext(), statusEntries);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import org.apache.nifi.controller.status.ConnectionStatus;
|
||||
import org.apache.nifi.controller.status.history.ComponentDetailsStorage;
|
||||
import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class QuestDbConnectionStatusStorage extends QuestDbComponentStatusStorage<ConnectionStatus> {
|
||||
private static final Map<Integer, MetricDescriptor<ConnectionStatus>> METRICS = new HashMap<>();
|
||||
|
||||
static {
|
||||
METRICS.put(2, ConnectionStatusDescriptor.INPUT_BYTES.getDescriptor());
|
||||
METRICS.put(3, ConnectionStatusDescriptor.INPUT_COUNT.getDescriptor());
|
||||
METRICS.put(4, ConnectionStatusDescriptor.OUTPUT_BYTES.getDescriptor());
|
||||
METRICS.put(5, ConnectionStatusDescriptor.OUTPUT_COUNT.getDescriptor());
|
||||
METRICS.put(6, ConnectionStatusDescriptor.QUEUED_BYTES.getDescriptor());
|
||||
METRICS.put(7, ConnectionStatusDescriptor.QUEUED_COUNT.getDescriptor());
|
||||
}
|
||||
|
||||
public QuestDbConnectionStatusStorage(final QuestDbContext dbContext, final ComponentDetailsStorage componentDetails) {
|
||||
super(dbContext, componentDetails);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String extractId(final ConnectionStatus statusEntry) {
|
||||
return statusEntry.getId();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<Integer, MetricDescriptor<ConnectionStatus>> getMetrics() {
|
||||
return METRICS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTableName() {
|
||||
return "connectionStatus";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.nifi.controller.status.history.GarbageCollectionHistory;
|
||||
import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
|
||||
import org.apache.nifi.controller.status.history.StandardGarbageCollectionHistory;
|
||||
import org.apache.nifi.controller.status.history.StandardGarbageCollectionStatus;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbEntityReadingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbEntityWritingTemplate;
|
||||
import org.apache.nifi.controller.status.history.storage.GarbageCollectionStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.StatusStorage;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
public class QuestDbGarbageCollectionStatusStorage implements GarbageCollectionStatusStorage {
|
||||
private static final String TABLE_NAME = "garbageCollectionStatus";
|
||||
|
||||
private static final String QUERY =
|
||||
"SELECT * FROM garbageCollectionStatus " +
|
||||
"WHERE capturedAt > to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"AND capturedAt < to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"ORDER BY capturedAt ASC";
|
||||
|
||||
private static final QuestDbEntityWritingTemplate<GarbageCollectionStatus> WRITING_TEMPLATE = new QuestDbEntityWritingTemplate<>(
|
||||
TABLE_NAME,
|
||||
(statusEntry, row) -> {
|
||||
row.putSym(1, statusEntry.getMemoryManagerName());
|
||||
row.putLong(2, statusEntry.getCollectionCount());
|
||||
row.putLong(3, statusEntry.getCollectionMillis());
|
||||
});
|
||||
|
||||
|
||||
private static final QuestDbEntityReadingTemplate<GarbageCollectionStatus, GarbageCollectionHistory> READING_TEMPLATE = new QuestDbEntityReadingTemplate<>(
|
||||
QUERY,
|
||||
record ->
|
||||
new StandardGarbageCollectionStatus(new StringBuilder(record.getSym(1)).toString(), new Date(record.getTimestamp(0)), record.getLong(2), record.getLong(3)),
|
||||
garbageCollectionStatuses -> {
|
||||
final StandardGarbageCollectionHistory result = new StandardGarbageCollectionHistory();
|
||||
garbageCollectionStatuses.forEach(status -> result.addGarbageCollectionStatus(status));
|
||||
return result;
|
||||
},
|
||||
e -> new StandardGarbageCollectionHistory()
|
||||
);
|
||||
|
||||
private final QuestDbContext context;
|
||||
|
||||
public QuestDbGarbageCollectionStatusStorage(final QuestDbContext context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public GarbageCollectionHistory read(final Instant start, final Instant end) {
|
||||
return READING_TEMPLATE.read(context.getEngine(), context.getSqlExecutionContext(), Arrays.asList(DATE_FORMATTER.format(start), DATE_FORMATTER.format(end)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void store(final List<Pair<Instant, GarbageCollectionStatus>> statusEntries) {
|
||||
WRITING_TEMPLATE.insert(context.getEngine(), context.getSqlExecutionContext(), statusEntries);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,141 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import io.questdb.cairo.sql.RecordCursor;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.NodeStatusDescriptor;
|
||||
import org.apache.nifi.controller.status.history.StandardMetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.StandardStatusHistory;
|
||||
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
|
||||
import org.apache.nifi.controller.status.history.StatusHistory;
|
||||
import org.apache.nifi.controller.status.history.StatusSnapshot;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbEntityWritingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbReadingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbWritingTemplate;
|
||||
import org.apache.nifi.controller.status.history.storage.NodeStatusStorage;
|
||||
import org.apache.nifi.controller.status.history.storage.StatusStorage;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class QuestDbNodeStatusStorage implements NodeStatusStorage {
|
||||
private static final String TABLE_NAME = "nodeStatus";
|
||||
|
||||
private static final String READING_QUERY =
|
||||
"SELECT * FROM nodeStatus " +
|
||||
"WHERE capturedAt > to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"AND capturedAt < to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"ORDER BY capturedAt ASC";
|
||||
|
||||
private static final Map<Integer, MetricDescriptor<NodeStatus>> METRICS = new HashMap<>();
|
||||
|
||||
private final QuestDbContext dbContext;
|
||||
|
||||
static {
|
||||
METRICS.put(1, NodeStatusDescriptor.FREE_HEAP.getDescriptor());
|
||||
METRICS.put(2, NodeStatusDescriptor.USED_HEAP.getDescriptor());
|
||||
METRICS.put(3, NodeStatusDescriptor.HEAP_UTILIZATION.getDescriptor());
|
||||
METRICS.put(4, NodeStatusDescriptor.FREE_NON_HEAP.getDescriptor());
|
||||
METRICS.put(5, NodeStatusDescriptor.USED_NON_HEAP.getDescriptor());
|
||||
METRICS.put(6, NodeStatusDescriptor.OPEN_FILE_HANDLES.getDescriptor());
|
||||
METRICS.put(7, NodeStatusDescriptor.PROCESSOR_LOAD_AVERAGE.getDescriptor());
|
||||
METRICS.put(8, NodeStatusDescriptor.TOTAL_THREADS.getDescriptor());
|
||||
METRICS.put(9, NodeStatusDescriptor.EVENT_DRIVEN_THREADS.getDescriptor());
|
||||
METRICS.put(10, NodeStatusDescriptor.TIME_DRIVEN_THREADS.getDescriptor());
|
||||
}
|
||||
|
||||
private static final QuestDbEntityWritingTemplate<NodeStatus> WRITING_TEMPLATE
|
||||
= new QuestDbEntityWritingTemplate<>(TABLE_NAME, (statusEntry, row) -> METRICS.keySet().forEach(ord -> row.putLong(ord, METRICS.get(ord).getValueFunction().getValue(statusEntry))));
|
||||
|
||||
private static final StorageStatusReadingTemplate STORAGE_READING_TEMPLATE = new StorageStatusReadingTemplate();
|
||||
|
||||
private static final QuestDbWritingTemplate<Pair<Instant, NodeStatus>> STORAGE_WRITING_TEMPLATE = new StorageStatusWritingTemplate();
|
||||
|
||||
public QuestDbNodeStatusStorage(final QuestDbContext dbContext) {
|
||||
this.dbContext = dbContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory read(final Instant start, final Instant end) {
|
||||
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
|
||||
final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> storageMetrics = STORAGE_READING_TEMPLATE
|
||||
.read(dbContext.getEngine(), executionContext, Arrays.asList(DATE_FORMATTER.format(start), DATE_FORMATTER.format(end)));
|
||||
final NodeStatusReadingTemplate nodeStatusReadingTemplate = new NodeStatusReadingTemplate(storageMetrics);
|
||||
final List<StatusSnapshot> snapshots = nodeStatusReadingTemplate
|
||||
.read(dbContext.getEngine(), executionContext, Arrays.asList(DATE_FORMATTER.format(start), DATE_FORMATTER.format(end)));
|
||||
return new StandardStatusHistory(snapshots, new HashMap<>(), new Date());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void store(final List<Pair<Instant, NodeStatus>> statusEntries) {
|
||||
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
|
||||
WRITING_TEMPLATE.insert(dbContext.getEngine(), executionContext, statusEntries);
|
||||
STORAGE_WRITING_TEMPLATE.insert(dbContext.getEngine(), executionContext, statusEntries);
|
||||
}
|
||||
|
||||
public static Map<Integer, MetricDescriptor<NodeStatus>> getMetrics() {
|
||||
return METRICS;
|
||||
}
|
||||
|
||||
private static class NodeStatusReadingTemplate extends QuestDbReadingTemplate<List<StatusSnapshot>> {
|
||||
private final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> storageMetricsByTime;
|
||||
|
||||
public NodeStatusReadingTemplate(final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> storageMetricsByTime) {
|
||||
super(READING_QUERY, e -> Collections.emptyList());
|
||||
this.storageMetricsByTime = storageMetricsByTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<StatusSnapshot> processResult(final RecordCursor cursor) {
|
||||
final List<StatusSnapshot> entities = new LinkedList<>();
|
||||
|
||||
while (cursor.hasNext()) {
|
||||
entities.add(map(cursor.getRecord()));
|
||||
}
|
||||
|
||||
return entities;
|
||||
}
|
||||
|
||||
private StandardStatusSnapshot map(final Record record) {
|
||||
final long createdAt = TimeUnit.MICROSECONDS.toMillis(record.getTimestamp(0));
|
||||
final Map<StandardMetricDescriptor<NodeStatus>, Long> statusMetrics = storageMetricsByTime.get(createdAt);
|
||||
final Set<MetricDescriptor<?>> snapshotMetrics = new HashSet<>(METRICS.values().size() + statusMetrics.keySet().size());
|
||||
snapshotMetrics.addAll(METRICS.values());
|
||||
snapshotMetrics.addAll(statusMetrics.keySet());
|
||||
final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(snapshotMetrics);
|
||||
snapshot.setTimestamp(new Date(createdAt));
|
||||
METRICS.keySet().forEach(ordinal -> snapshot.addStatusMetric(METRICS.get(ordinal), record.getLong(ordinal)));
|
||||
statusMetrics.entrySet().forEach(entry -> snapshot.addStatusMetric(entry.getKey(), entry.getValue()));
|
||||
return snapshot;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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.history.storage.questdb;
|
||||
|
||||
import org.apache.nifi.controller.status.ProcessGroupStatus;
|
||||
import org.apache.nifi.controller.status.history.ComponentDetailsStorage;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.ProcessGroupStatusDescriptor;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class QuestDbProcessGroupStatusStorage extends QuestDbComponentStatusStorage<ProcessGroupStatus> {
|
||||
private static final Map<Integer, MetricDescriptor<ProcessGroupStatus>> METRICS = new HashMap<>();
|
||||
|
||||
static {
|
||||
METRICS.put(2, ProcessGroupStatusDescriptor.BYTES_READ.getDescriptor());
|
||||
METRICS.put(3, ProcessGroupStatusDescriptor.BYTES_WRITTEN.getDescriptor());
|
||||
METRICS.put(4, ProcessGroupStatusDescriptor.BYTES_TRANSFERRED.getDescriptor());
|
||||
METRICS.put(5, ProcessGroupStatusDescriptor.INPUT_BYTES.getDescriptor());
|
||||
METRICS.put(6, ProcessGroupStatusDescriptor.INPUT_COUNT.getDescriptor());
|
||||
METRICS.put(7, ProcessGroupStatusDescriptor.OUTPUT_BYTES.getDescriptor());
|
||||
METRICS.put(8, ProcessGroupStatusDescriptor.OUTPUT_COUNT.getDescriptor());
|
||||
METRICS.put(9, ProcessGroupStatusDescriptor.QUEUED_BYTES.getDescriptor());
|
||||
METRICS.put(10, ProcessGroupStatusDescriptor.QUEUED_COUNT.getDescriptor());
|
||||
METRICS.put(11, ProcessGroupStatusDescriptor.TASK_MILLIS.getDescriptor());
|
||||
}
|
||||
|
||||
public QuestDbProcessGroupStatusStorage(final QuestDbContext dbContext, final ComponentDetailsStorage componentDetailsStorage) {
|
||||
super(dbContext, componentDetailsStorage);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String extractId(final ProcessGroupStatus statusEntry) {
|
||||
return statusEntry.getId();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<Integer, MetricDescriptor<ProcessGroupStatus>> getMetrics() {
|
||||
return METRICS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTableName() {
|
||||
return "processGroupStatus";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,167 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import io.questdb.cairo.sql.RecordCursor;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.nifi.controller.status.ProcessorStatus;
|
||||
import org.apache.nifi.controller.status.history.ComponentDetailsStorage;
|
||||
import org.apache.nifi.controller.status.history.CounterMetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.ProcessorStatusDescriptor;
|
||||
import org.apache.nifi.controller.status.history.StandardStatusHistory;
|
||||
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
|
||||
import org.apache.nifi.controller.status.history.StatusHistory;
|
||||
import org.apache.nifi.controller.status.history.StatusSnapshot;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbEntityReadingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbEntityWritingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbReadingTemplate;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbStatusSnapshotMapper;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbWritingTemplate;
|
||||
import org.apache.nifi.controller.status.history.storage.ProcessorStatusStorage;
|
||||
|
||||
import java.time.Instant;
|
||||
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.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class QuestDbProcessorStatusStorage implements ProcessorStatusStorage {
|
||||
private static final String TABLE_NAME = "processorStatus";
|
||||
private static final Map<Integer, MetricDescriptor<ProcessorStatus>> METRICS = new HashMap<>();
|
||||
|
||||
static {
|
||||
METRICS.put(2, ProcessorStatusDescriptor.BYTES_READ.getDescriptor());
|
||||
METRICS.put(3, ProcessorStatusDescriptor.BYTES_WRITTEN.getDescriptor());
|
||||
METRICS.put(4, ProcessorStatusDescriptor.BYTES_TRANSFERRED.getDescriptor());
|
||||
METRICS.put(5, ProcessorStatusDescriptor.INPUT_BYTES.getDescriptor());
|
||||
METRICS.put(6, ProcessorStatusDescriptor.INPUT_COUNT.getDescriptor());
|
||||
METRICS.put(7, ProcessorStatusDescriptor.OUTPUT_BYTES.getDescriptor());
|
||||
METRICS.put(8, ProcessorStatusDescriptor.OUTPUT_COUNT.getDescriptor());
|
||||
METRICS.put(9, ProcessorStatusDescriptor.TASK_COUNT.getDescriptor());
|
||||
METRICS.put(10, ProcessorStatusDescriptor.TASK_MILLIS.getDescriptor());
|
||||
METRICS.put(11, ProcessorStatusDescriptor.TASK_NANOS.getDescriptor());
|
||||
METRICS.put(12, ProcessorStatusDescriptor.FLOWFILES_REMOVED.getDescriptor());
|
||||
METRICS.put(13, ProcessorStatusDescriptor.AVERAGE_LINEAGE_DURATION.getDescriptor());
|
||||
METRICS.put(14, ProcessorStatusDescriptor.AVERAGE_TASK_NANOS.getDescriptor());
|
||||
}
|
||||
|
||||
/**
|
||||
* In case of component status entries the first two columns are fixed (measurement time and component id) and all
|
||||
* the following fields are metric values using long format. The list of these are specified by the implementation class.
|
||||
*/
|
||||
private final QuestDbEntityWritingTemplate<ProcessorStatus> writingTemplate = new QuestDbEntityWritingTemplate<>(
|
||||
TABLE_NAME,
|
||||
(statusEntry, row) -> {
|
||||
row.putSym(1, statusEntry.getId());
|
||||
METRICS.keySet().forEach(ordinal -> row.putLong(ordinal, METRICS.get(ordinal).getValueFunction().getValue(statusEntry)));
|
||||
});
|
||||
|
||||
private static final QuestDbWritingTemplate<Pair<Instant, ProcessorStatus>> counterWritingTemplate = new ComponentCounterWritingTemplate();
|
||||
|
||||
private final Function<Record, StandardStatusSnapshot> statusSnapshotMapper = new QuestDbStatusSnapshotMapper(METRICS);
|
||||
|
||||
private final QuestDbEntityReadingTemplate<StandardStatusSnapshot, List<StandardStatusSnapshot>> readingTemplate
|
||||
= new QuestDbEntityReadingTemplate<>(QUERY_TEMPLATE, statusSnapshotMapper, e -> e, e -> Collections.emptyList());
|
||||
|
||||
private final QuestDbContext dbContext;
|
||||
private final ComponentDetailsStorage componentDetailsStorage;
|
||||
|
||||
public QuestDbProcessorStatusStorage(final QuestDbContext dbContext, final ComponentDetailsStorage componentDetailsStorage) {
|
||||
this.dbContext = dbContext;
|
||||
this.componentDetailsStorage = componentDetailsStorage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory read(final String componentId, final Instant start, final Instant end, final int preferredDataPoints) {
|
||||
final List<StandardStatusSnapshot> snapshots = readingTemplate.read(
|
||||
dbContext.getEngine(),
|
||||
dbContext.getSqlExecutionContext(),
|
||||
Arrays.asList(TABLE_NAME, componentId, DATE_FORMATTER.format(start), DATE_FORMATTER.format(end)));
|
||||
return new StandardStatusHistory(
|
||||
new ArrayList<>(snapshots.subList(Math.max(snapshots.size() - preferredDataPoints, 0), snapshots.size())),
|
||||
componentDetailsStorage.getDetails(componentId),
|
||||
new Date()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StatusHistory readWithCounter(final String componentId, final Instant start, final Instant end, final int preferredDataPoints) {
|
||||
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
|
||||
final List<StandardStatusSnapshot> snapshots = readingTemplate.read(
|
||||
dbContext.getEngine(),
|
||||
executionContext,
|
||||
Arrays.asList(TABLE_NAME, componentId, DATE_FORMATTER.format(start), DATE_FORMATTER.format(end)));
|
||||
final CounterReadingTemplate counterReadingTemplate = new CounterReadingTemplate(snapshots);
|
||||
final List<StatusSnapshot> enrichedSnapshots = new ArrayList<>(counterReadingTemplate.read(
|
||||
dbContext.getEngine(),
|
||||
executionContext,
|
||||
Arrays.asList("componentCounter", componentId, DATE_FORMATTER.format(start), DATE_FORMATTER.format(end))));
|
||||
return new StandardStatusHistory(
|
||||
enrichedSnapshots.subList(Math.max(snapshots.size() - preferredDataPoints, 0), snapshots.size()),
|
||||
componentDetailsStorage.getDetails(componentId),
|
||||
new Date()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void store(final List<Pair<Instant, ProcessorStatus>> statusEntries) {
|
||||
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
|
||||
writingTemplate.insert(dbContext.getEngine(), executionContext, statusEntries);
|
||||
counterWritingTemplate.insert(dbContext.getEngine(), executionContext, statusEntries);
|
||||
}
|
||||
|
||||
private final class CounterReadingTemplate extends QuestDbReadingTemplate<List<StandardStatusSnapshot>> {
|
||||
private final List<StandardStatusSnapshot> processorStatusSnapshots;
|
||||
|
||||
public CounterReadingTemplate(final List<StandardStatusSnapshot> processorStatusSnapshots) {
|
||||
super(QUERY_TEMPLATE, e -> Collections.emptyList());
|
||||
this.processorStatusSnapshots = processorStatusSnapshots;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<StandardStatusSnapshot> processResult(final RecordCursor cursor) {
|
||||
final Map<Long, StandardStatusSnapshot> snapshotsByTime = processorStatusSnapshots.stream().collect(Collectors.toMap(s -> s.getTimestamp().getTime(), s -> s));
|
||||
|
||||
while (cursor.hasNext()) {
|
||||
final Record record = cursor.getRecord();
|
||||
final long recordCreatedAt = TimeUnit.MICROSECONDS.toMillis(record.getTimestamp(0));
|
||||
final StandardStatusSnapshot snapshot = snapshotsByTime.get(recordCreatedAt);
|
||||
final String counterName = new StringBuilder(record.getSym(2)).toString();
|
||||
final long counterValue = record.getLong(3);
|
||||
final MetricDescriptor<ProcessorStatus> metricDescriptor = new CounterMetricDescriptor<>(
|
||||
counterName,
|
||||
counterName + " (5 mins)",
|
||||
counterName + " (5 mins)",
|
||||
MetricDescriptor.Formatter.COUNT,
|
||||
s -> s.getCounters() == null ? null : s.getCounters().get(counterName));
|
||||
snapshot.addStatusMetric(metricDescriptor, counterValue);
|
||||
}
|
||||
|
||||
return processorStatusSnapshots;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
|
||||
import org.apache.nifi.controller.status.history.ComponentDetailsStorage;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.RemoteProcessGroupStatusDescriptor;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class QuestDbRemoteProcessGroupStatusStorage extends QuestDbComponentStatusStorage<RemoteProcessGroupStatus> {
|
||||
private static final Map<Integer, MetricDescriptor<RemoteProcessGroupStatus>> METRICS = new HashMap<>();
|
||||
|
||||
static {
|
||||
METRICS.put(2, RemoteProcessGroupStatusDescriptor.SENT_BYTES.getDescriptor());
|
||||
METRICS.put(3, RemoteProcessGroupStatusDescriptor.SENT_COUNT.getDescriptor());
|
||||
METRICS.put(4, RemoteProcessGroupStatusDescriptor.RECEIVED_BYTES.getDescriptor());
|
||||
METRICS.put(5, RemoteProcessGroupStatusDescriptor.RECEIVED_COUNT.getDescriptor());
|
||||
METRICS.put(6, RemoteProcessGroupStatusDescriptor.RECEIVED_BYTES_PER_SECOND.getDescriptor());
|
||||
METRICS.put(7, RemoteProcessGroupStatusDescriptor.SENT_BYTES_PER_SECOND.getDescriptor());
|
||||
METRICS.put(8, RemoteProcessGroupStatusDescriptor.TOTAL_BYTES_PER_SECOND.getDescriptor());
|
||||
METRICS.put(9, RemoteProcessGroupStatusDescriptor.AVERAGE_LINEAGE_DURATION.getDescriptor());
|
||||
}
|
||||
|
||||
public QuestDbRemoteProcessGroupStatusStorage(final QuestDbContext context, final ComponentDetailsStorage componentDetailsStorage) {
|
||||
super(context, componentDetailsStorage);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String extractId(final RemoteProcessGroupStatus statusEntry) {
|
||||
return statusEntry.getId();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<Integer, MetricDescriptor<RemoteProcessGroupStatus>> getMetrics() {
|
||||
return METRICS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTableName() {
|
||||
return "remoteProcessGroupStatus";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,136 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import io.questdb.cairo.sql.RecordCursor;
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.apache.nifi.controller.status.history.MetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.StandardMetricDescriptor;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbReadingTemplate;
|
||||
import org.apache.nifi.controller.status.history.storage.StatusStorage;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class StorageStatusReadingTemplate extends QuestDbReadingTemplate<Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>>> {
|
||||
private static final String STORAGE_FREE_DESCRIPTION = "The usable space available for use by the underlying storage mechanism.";
|
||||
private static final String STORAGE_USED_DESCRIPTION = "The space in use on the underlying storage mechanism";
|
||||
|
||||
private static final String STORAGE_READING_QUERY =
|
||||
"SELECT * FROM storageStatus " +
|
||||
"WHERE capturedAt > to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"AND capturedAt < to_timestamp('%s', '" + StatusStorage.CAPTURE_DATE_FORMAT + "') " +
|
||||
"ORDER BY capturedAt ASC";
|
||||
|
||||
public StorageStatusReadingTemplate() {
|
||||
super(STORAGE_READING_QUERY, e -> Collections.emptyMap());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> processResult(final RecordCursor cursor) {
|
||||
final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> result = new HashMap<>();
|
||||
|
||||
int storageNumber = 1;
|
||||
|
||||
while (cursor.hasNext()) {
|
||||
final Record record = cursor.getRecord();
|
||||
final long createdAt = TimeUnit.MICROSECONDS.toMillis(record.getTimestamp(0));
|
||||
final short type = record.getShort(2);
|
||||
final CharSequence name = record.getSym(1);
|
||||
|
||||
if (!result.containsKey(createdAt)) {
|
||||
result.put(createdAt, new HashMap<>());
|
||||
}
|
||||
|
||||
result.get(createdAt).put(getDescriptor(
|
||||
QuestDbNodeStatusStorage.getMetrics().size() + result.get(createdAt).size(),
|
||||
getField(type, storageNumber, StorageMetric.FREE),
|
||||
getLabel(type, name, StorageMetric.FREE),
|
||||
STORAGE_FREE_DESCRIPTION
|
||||
), record.getLong(3));
|
||||
result.get(createdAt).put(getDescriptor(
|
||||
QuestDbNodeStatusStorage.getMetrics().size() + result.get(createdAt).size(),
|
||||
getField(type, storageNumber, StorageMetric.USED),
|
||||
getLabel(type, name, StorageMetric.USED),
|
||||
STORAGE_USED_DESCRIPTION
|
||||
), record.getLong(4));
|
||||
storageNumber++;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private StandardMetricDescriptor<NodeStatus> getDescriptor(final int ordinal, final String field, final String label, final String description) {
|
||||
return new StandardMetricDescriptor<>(() -> ordinal, field, label, STORAGE_FREE_DESCRIPTION, MetricDescriptor.Formatter.DATA_SIZE, v -> 0L);
|
||||
}
|
||||
|
||||
private String getField(final int type, final int storageNumber, final StorageMetric storageMetric) {
|
||||
return new StringBuilder(StorageType.getById(type).getField()).append(storageNumber).append(storageMetric.getField()).toString();
|
||||
}
|
||||
|
||||
private String getLabel(final int type, final CharSequence name, final StorageMetric storageMetric) {
|
||||
return new StringBuilder(StorageType.getById(type).getLabel()).append(" (").append(name).append(") ").append(storageMetric.getLabel()).toString();
|
||||
}
|
||||
|
||||
private enum StorageType {
|
||||
CONTENT("contentStorage", "Content Repository"), // 0
|
||||
PROVENANCE("provenanceStorage", "Provenance Repository"); // 1
|
||||
|
||||
private final String field;
|
||||
private final String label;
|
||||
|
||||
StorageType(final String field, final String label) {
|
||||
this.field = field;
|
||||
this.label = label;
|
||||
}
|
||||
|
||||
public static StorageType getById(final int id) {
|
||||
return StorageType.values()[id];
|
||||
}
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public String getLabel() {
|
||||
return label;
|
||||
}
|
||||
}
|
||||
|
||||
private enum StorageMetric {
|
||||
FREE("Free", "Free Space"), USED("Used", "Used Space");
|
||||
|
||||
private final String field;
|
||||
private final String label;
|
||||
|
||||
StorageMetric(final String field, final String label) {
|
||||
this.field = field;
|
||||
this.label = label;
|
||||
}
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public String getLabel() {
|
||||
return label;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage.questdb;
|
||||
|
||||
import io.questdb.cairo.TableWriter;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.apache.nifi.controller.status.StorageStatus;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbWritingTemplate;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
class StorageStatusWritingTemplate extends QuestDbWritingTemplate<Pair<Instant, NodeStatus>> {
|
||||
|
||||
public StorageStatusWritingTemplate() {
|
||||
super("storageStatus");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void addRows(final TableWriter tableWriter, final Collection<Pair<Instant, NodeStatus>> entries) {
|
||||
for (final Pair<Instant, NodeStatus> entry : entries) {
|
||||
for (final StorageStatus contentRepository : entry.getRight().getContentRepositories()) {
|
||||
final long capturedAt = TimeUnit.MILLISECONDS.toMicros(entry.getLeft().toEpochMilli());
|
||||
final TableWriter.Row row = tableWriter.newRow(capturedAt);
|
||||
row.putTimestamp(0, capturedAt);
|
||||
row.putSym(1, contentRepository.getName());
|
||||
row.putShort(2, Integer.valueOf(0).shortValue());
|
||||
row.putLong(3, contentRepository.getFreeSpace());
|
||||
row.putLong(4, contentRepository.getUsedSpace());
|
||||
row.append();
|
||||
}
|
||||
|
||||
for (final StorageStatus provenanceRepository : entry.getRight().getProvenanceRepositories()) {
|
||||
final long capturedAt = TimeUnit.MILLISECONDS.toMicros(entry.getLeft().toEpochMilli());
|
||||
final TableWriter.Row row = tableWriter.newRow(capturedAt);
|
||||
row.putTimestamp(0, capturedAt);
|
||||
row.putSym(1, provenanceRepository.getName());
|
||||
row.putShort(2, Integer.valueOf(1).shortValue());
|
||||
row.putLong(3, provenanceRepository.getFreeSpace());
|
||||
row.putLong(4, provenanceRepository.getUsedSpace());
|
||||
row.append();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -12,4 +12,5 @@
|
|||
# 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.history.VolatileComponentStatusRepository
|
||||
org.apache.nifi.controller.status.history.VolatileComponentStatusRepository
|
||||
org.apache.nifi.controller.status.history.EmbeddedQuestDbStatusHistoryRepository
|
|
@ -20,14 +20,14 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertNotEquals;
|
||||
|
||||
import org.apache.nifi.controller.flow.FlowManager;
|
||||
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestCachingConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
|
||||
|
||||
@Override
|
||||
public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager,
|
||||
ComponentStatusRepository componentStatusRepository, StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory,
|
||||
StatusHistoryRepository componentStatusRepository, StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory,
|
||||
long predictIntervalMillis, long queryIntervalMillis, String scoreName, double scoreThreshold) {
|
||||
|
||||
return new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, statusAnalyticsModelMapFactory, predictIntervalMillis,
|
||||
|
|
|
@ -42,10 +42,10 @@ 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.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.controller.status.history.StatusHistoryRepository;
|
||||
import org.apache.nifi.groups.ProcessGroup;
|
||||
import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
|
||||
import org.apache.nifi.nar.SystemBundle;
|
||||
|
@ -68,7 +68,7 @@ public class TestConnectionStatusAnalytics {
|
|||
|
||||
protected ConnectionStatusAnalytics getConnectionStatusAnalytics(Map<String, Tuple<StatusAnalyticsModel, StatusMetricExtractFunction>> modelMap) {
|
||||
|
||||
ComponentStatusRepository statusRepository = Mockito.mock(ComponentStatusRepository.class);
|
||||
StatusHistoryRepository statusRepository = Mockito.mock(StatusHistoryRepository.class);
|
||||
FlowManager flowManager;
|
||||
flowManager = Mockito.mock(FlowManager.class);
|
||||
final Map<String, String> otherProps = new HashMap<>();
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
package org.apache.nifi.controller.status.analytics;
|
||||
|
||||
import org.apache.nifi.controller.flow.FlowManager;
|
||||
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
|
||||
public class TestConnectionStatusAnalyticsEngine extends TestStatusAnalyticsEngine {
|
||||
|
||||
@Override
|
||||
public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, ComponentStatusRepository statusRepository,
|
||||
public StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager, StatusHistoryRepository statusRepository,
|
||||
StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory,
|
||||
long predictIntervalMillis, long queryIntervalMillis, String scoreName, double scoreThreshold) {
|
||||
return new ConnectionStatusAnalyticsEngine(flowManager, statusRepository, statusAnalyticsModelMapFactory,
|
||||
|
|
|
@ -29,8 +29,8 @@ 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.StatusHistoryRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusSnapshot;
|
||||
import org.apache.nifi.groups.ProcessGroup;
|
||||
import org.apache.nifi.util.Tuple;
|
||||
|
@ -47,7 +47,7 @@ public abstract class TestStatusAnalyticsEngine {
|
|||
static final String DEFAULT_SCORE_NAME = "rSquared";
|
||||
static final double DEFAULT_SCORE_THRESHOLD = .9;
|
||||
|
||||
protected ComponentStatusRepository statusRepository;
|
||||
protected StatusHistoryRepository statusRepository;
|
||||
protected FlowManager flowManager;
|
||||
protected FlowFileEventRepository flowFileEventRepository;
|
||||
protected StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory;
|
||||
|
@ -55,7 +55,7 @@ public abstract class TestStatusAnalyticsEngine {
|
|||
@Before
|
||||
public void setup() {
|
||||
|
||||
statusRepository = Mockito.mock(ComponentStatusRepository.class);
|
||||
statusRepository = Mockito.mock(StatusHistoryRepository.class);
|
||||
flowManager = Mockito.mock(FlowManager.class);
|
||||
statusAnalyticsModelMapFactory = Mockito.mock(StatusAnalyticsModelMapFactory.class);
|
||||
|
||||
|
@ -99,7 +99,7 @@ public abstract class TestStatusAnalyticsEngine {
|
|||
}
|
||||
|
||||
public abstract StatusAnalyticsEngine getStatusAnalyticsEngine(FlowManager flowManager,
|
||||
ComponentStatusRepository componentStatusRepository, StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory,
|
||||
StatusHistoryRepository componentStatusRepository, StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory,
|
||||
long predictIntervalMillis, long queryIntervalMillis, String scoreName, double scoreThreshold);
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import org.apache.nifi.util.FileUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.mockito.Mockito;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Date;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public abstract class AbstractEmbeddedQuestDbStatusHistoryRepositoryTest extends AbstractStatusHistoryRepositoryTest {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.class);
|
||||
|
||||
protected static final String PATH = "target/questdb";
|
||||
protected static final long NOW = System.currentTimeMillis();
|
||||
protected static final Date START = new Date(0);
|
||||
protected static final Date INSERTED_AT = new Date(NOW - TimeUnit.MINUTES.toMillis(1));
|
||||
protected static final Date END = new Date(NOW);
|
||||
protected static final Date END_EARLY = new Date(NOW - TimeUnit.MINUTES.toMillis(10));
|
||||
|
||||
protected static final int PREFERRED_DATA_POINTS = 1000;
|
||||
protected static final int DAYS_TO_KEEP_DATA = 7;
|
||||
protected static final long PERSIST_FREQUENCY = TimeUnit.MILLISECONDS.convert(2, TimeUnit.SECONDS);
|
||||
|
||||
protected EmbeddedQuestDbStatusHistoryRepository testSubject;
|
||||
protected String path;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
path = PATH + System.currentTimeMillis();
|
||||
testSubject = givenTestSubject();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
testSubject.shutdown();
|
||||
|
||||
try {
|
||||
FileUtils.deleteFile(new File(path), true);
|
||||
} catch (final Exception e) {
|
||||
LOGGER.error("Could not delete database directory", e);
|
||||
}
|
||||
}
|
||||
|
||||
private EmbeddedQuestDbStatusHistoryRepository givenTestSubject() {
|
||||
final NiFiProperties niFiProperties = Mockito.mock(NiFiProperties.class);
|
||||
|
||||
Mockito.when(niFiProperties.getIntegerProperty(
|
||||
NiFiProperties.STATUS_REPOSITORY_QUESTDB_PERSIST_NODE_DAYS,
|
||||
NiFiProperties.DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_NODE_DAYS)
|
||||
).thenReturn(DAYS_TO_KEEP_DATA);
|
||||
|
||||
Mockito.when(niFiProperties.getIntegerProperty(
|
||||
NiFiProperties.STATUS_REPOSITORY_QUESTDB_PERSIST_COMPONENT_DAYS,
|
||||
NiFiProperties.DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_COMPONENT_DAYS)
|
||||
).thenReturn(DAYS_TO_KEEP_DATA);
|
||||
|
||||
Mockito.when(niFiProperties.getQuestDbStatusRepositoryPath()).thenReturn(Paths.get(path));
|
||||
|
||||
final EmbeddedQuestDbStatusHistoryRepository testSubject = new EmbeddedQuestDbStatusHistoryRepository(niFiProperties, PERSIST_FREQUENCY);
|
||||
testSubject.start();
|
||||
return testSubject;
|
||||
}
|
||||
|
||||
protected void givenWaitUntilPersisted() throws InterruptedException {
|
||||
Thread.sleep(3000); // The actual writing happens asynchronously on a different thread
|
||||
}
|
||||
}
|
|
@ -0,0 +1,387 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import org.apache.nifi.controller.status.ConnectionStatus;
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
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.StorageStatus;
|
||||
import org.junit.Assert;
|
||||
|
||||
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.stream.Collectors;
|
||||
|
||||
public abstract class AbstractStatusHistoryRepositoryTest {
|
||||
protected static final String ROOT_GROUP_ID = "697199b5-8318-4004-8e18-75c6e882f29e";
|
||||
protected static final String CHILD_GROUP_ID = "fc4aef2e-84d6-4007-9e67-c53050d22398";
|
||||
protected static final String PROCESSOR_ID = "3ab6181b-925c-4803-8f72-ea1fdc065db8";
|
||||
protected static final String PROCESSOR_WITH_COUNTER_ID = "f09c5bb8-7b67-4d3b-81c5-5373e1c03ed1";
|
||||
protected static final String CONNECTION_ID = "d5452f70-a0d9-44c5-aeda-c2026482e4ee";
|
||||
protected static final String REMOTE_PROCESS_GROUP_ID = "f5f4fe2a-0209-4ba7-8f15-f33df942cde5";
|
||||
|
||||
protected ProcessGroupStatus givenSimpleRootProcessGroupStatus() {
|
||||
final ProcessGroupStatus status = new ProcessGroupStatus();
|
||||
status.setId(ROOT_GROUP_ID);
|
||||
status.setName("Root");
|
||||
status.setInputCount(1);
|
||||
status.setInputContentSize(2L);
|
||||
status.setOutputCount(3);
|
||||
status.setOutputContentSize(4L);
|
||||
status.setActiveThreadCount(5);
|
||||
status.setTerminatedThreadCount(6);
|
||||
status.setQueuedCount(7);
|
||||
status.setQueuedContentSize(8L);
|
||||
status.setBytesRead(9L);
|
||||
status.setBytesWritten(10L);
|
||||
status.setFlowFilesReceived(11);
|
||||
status.setBytesReceived(12L);
|
||||
status.setFlowFilesSent(13);
|
||||
status.setBytesSent(14);
|
||||
status.setFlowFilesTransferred(15);
|
||||
status.setBytesTransferred(16L);
|
||||
return status;
|
||||
}
|
||||
|
||||
protected ProcessGroupStatus givenRootProcessGroupStatus() {
|
||||
final ProcessGroupStatus status = givenSimpleRootProcessGroupStatus();
|
||||
status.setConnectionStatus(Collections.singleton(givenConnectionStatus()));
|
||||
status.setProcessGroupStatus(Collections.singleton(givenChildProcessGroupStatus()));
|
||||
status.setProcessorStatus(Collections.singleton(givenProcessorStatus()));
|
||||
return status;
|
||||
}
|
||||
|
||||
protected ProcessGroupStatus givenChildProcessGroupStatus() {
|
||||
final ProcessGroupStatus status = new ProcessGroupStatus();
|
||||
status.setId(CHILD_GROUP_ID);
|
||||
status.setName("Child");
|
||||
status.setInputCount(21);
|
||||
status.setInputContentSize(22L);
|
||||
status.setOutputCount(23);
|
||||
status.setOutputContentSize(24L);
|
||||
status.setActiveThreadCount(25);
|
||||
status.setTerminatedThreadCount(26);
|
||||
status.setQueuedCount(27);
|
||||
status.setQueuedContentSize(28L);
|
||||
status.setBytesRead(29L);
|
||||
status.setBytesWritten(30L);
|
||||
status.setFlowFilesReceived(31);
|
||||
status.setBytesReceived(32L);
|
||||
status.setFlowFilesSent(33);
|
||||
status.setBytesSent(34);
|
||||
status.setFlowFilesTransferred(35);
|
||||
status.setBytesTransferred(36L);
|
||||
|
||||
status.setRemoteProcessGroupStatus(Collections.singleton(givenRemoteProcessGroupStatus()));
|
||||
status.setProcessorStatus(Collections.singleton(givenProcessorWithCounterStatus()));
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
protected ProcessorStatus givenProcessorStatus() {
|
||||
final ProcessorStatus status = new ProcessorStatus();
|
||||
status.setId(PROCESSOR_ID);
|
||||
status.setName("Processor");
|
||||
status.setInputCount(61);
|
||||
status.setInputBytes(62);
|
||||
status.setOutputCount(63);
|
||||
status.setOutputBytes(64);
|
||||
status.setBytesRead(65);
|
||||
status.setBytesWritten(66);
|
||||
status.setInvocations(67);
|
||||
status.setProcessingNanos(68000000);
|
||||
status.setFlowFilesRemoved(69);
|
||||
status.setAverageLineageDuration(70);
|
||||
status.setActiveThreadCount(71);
|
||||
status.setTerminatedThreadCount(72);
|
||||
status.setFlowFilesReceived(73);
|
||||
status.setBytesReceived(74);
|
||||
status.setFlowFilesSent(75);
|
||||
status.setBytesSent(76);
|
||||
return status;
|
||||
}
|
||||
|
||||
protected ProcessorStatus givenProcessorWithCounterStatus() {
|
||||
final Map<String, Long> counters = new HashMap<>();
|
||||
counters.put("counter1", 97L);
|
||||
counters.put("counter2", 98L);
|
||||
|
||||
final ProcessorStatus status = new ProcessorStatus();
|
||||
status.setId(PROCESSOR_WITH_COUNTER_ID);
|
||||
status.setName("ProcessorWithCounter");
|
||||
status.setInputCount(81);
|
||||
status.setInputBytes(82);
|
||||
status.setOutputCount(83);
|
||||
status.setOutputBytes(84);
|
||||
status.setBytesRead(85);
|
||||
status.setBytesWritten(86);
|
||||
status.setInvocations(87);
|
||||
status.setProcessingNanos(88000000);
|
||||
status.setFlowFilesRemoved(89);
|
||||
status.setAverageLineageDuration(90);
|
||||
status.setActiveThreadCount(91);
|
||||
status.setTerminatedThreadCount(92);
|
||||
status.setFlowFilesReceived(93);
|
||||
status.setBytesReceived(94);
|
||||
status.setFlowFilesSent(95);
|
||||
status.setBytesSent(96);
|
||||
status.setCounters(counters);
|
||||
return status;
|
||||
}
|
||||
|
||||
protected ConnectionStatus givenConnectionStatus() {
|
||||
final ConnectionStatus status = new ConnectionStatus();
|
||||
status.setId(CONNECTION_ID);
|
||||
status.setName("Connection");
|
||||
status.setInputCount(101);
|
||||
status.setInputBytes(102);
|
||||
status.setQueuedCount(103);
|
||||
status.setQueuedBytes(104);
|
||||
status.setOutputCount(105);
|
||||
status.setOutputBytes(106);
|
||||
status.setMaxQueuedCount(107);
|
||||
status.setMaxQueuedBytes(108);
|
||||
return status;
|
||||
}
|
||||
|
||||
protected RemoteProcessGroupStatus givenRemoteProcessGroupStatus() {
|
||||
final RemoteProcessGroupStatus status = new RemoteProcessGroupStatus();
|
||||
status.setId(REMOTE_PROCESS_GROUP_ID);
|
||||
status.setName("RemoteProcessGroup");
|
||||
status.setActiveThreadCount(121);
|
||||
status.setSentCount(122);
|
||||
status.setSentContentSize(123000L);
|
||||
status.setReceivedCount(124);
|
||||
status.setReceivedContentSize(125000L);
|
||||
status.setActiveRemotePortCount(126);
|
||||
status.setInactiveRemotePortCount(127);
|
||||
status.setAverageLineageDuration(128000);
|
||||
return status;
|
||||
}
|
||||
|
||||
protected void assertStatusHistoryIsEmpty(final StatusHistory statusHistory) {
|
||||
Assert.assertTrue(statusHistory.getStatusSnapshots().isEmpty());
|
||||
}
|
||||
|
||||
protected void assertRootProcessGroupStatusSnapshot(final StatusSnapshot snapshot) {
|
||||
Assert.assertEquals(9L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.BYTES_READ.getDescriptor()).longValue());
|
||||
Assert.assertEquals(10L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.BYTES_WRITTEN.getDescriptor()).longValue());
|
||||
Assert.assertEquals(9L+10L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.BYTES_TRANSFERRED.getDescriptor()).longValue());
|
||||
Assert.assertEquals(2L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.INPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(1L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.INPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(4L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.OUTPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(3L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.OUTPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(8L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.QUEUED_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(7L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.QUEUED_COUNT.getDescriptor()).longValue());
|
||||
|
||||
// Information is lost due to nano->micro conversion!
|
||||
Assert.assertEquals(Double.valueOf((68000000L + 88000000L)/1000/1000).longValue(), snapshot.getStatusMetric(ProcessGroupStatusDescriptor.TASK_MILLIS.getDescriptor()).longValue());
|
||||
}
|
||||
|
||||
protected void assertChildProcessGroupStatusSnapshot(final StatusSnapshot snapshot) {
|
||||
Assert.assertEquals(29L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.BYTES_READ.getDescriptor()).longValue());
|
||||
Assert.assertEquals(30L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.BYTES_WRITTEN.getDescriptor()).longValue());
|
||||
Assert.assertEquals(29L+30L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.BYTES_TRANSFERRED.getDescriptor()).longValue());
|
||||
Assert.assertEquals(22L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.INPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(21L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.INPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(24L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.OUTPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(23L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.OUTPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(28L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.QUEUED_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(27L, snapshot.getStatusMetric(ProcessGroupStatusDescriptor.QUEUED_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(Double.valueOf((88000000L)/1000/1000).longValue(), snapshot.getStatusMetric(ProcessGroupStatusDescriptor.TASK_MILLIS.getDescriptor()).longValue());
|
||||
}
|
||||
|
||||
protected void assertProcessorStatusSnapshot(final StatusSnapshot snapshot) {
|
||||
Assert.assertEquals(65L, snapshot.getStatusMetric(ProcessorStatusDescriptor.BYTES_READ.getDescriptor()).longValue());
|
||||
Assert.assertEquals(66L, snapshot.getStatusMetric(ProcessorStatusDescriptor.BYTES_WRITTEN.getDescriptor()).longValue());
|
||||
Assert.assertEquals(65L+66L, snapshot.getStatusMetric(ProcessorStatusDescriptor.BYTES_TRANSFERRED.getDescriptor()).longValue());
|
||||
Assert.assertEquals(62L, snapshot.getStatusMetric(ProcessorStatusDescriptor.INPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(61L, snapshot.getStatusMetric(ProcessorStatusDescriptor.INPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(64L, snapshot.getStatusMetric(ProcessorStatusDescriptor.OUTPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(63L, snapshot.getStatusMetric(ProcessorStatusDescriptor.OUTPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(67L, snapshot.getStatusMetric(ProcessorStatusDescriptor.TASK_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(68L, snapshot.getStatusMetric(ProcessorStatusDescriptor.TASK_MILLIS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(68000000L, snapshot.getStatusMetric(ProcessorStatusDescriptor.TASK_NANOS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(69L, snapshot.getStatusMetric(ProcessorStatusDescriptor.FLOWFILES_REMOVED.getDescriptor()).longValue());
|
||||
Assert.assertEquals(70L, snapshot.getStatusMetric(ProcessorStatusDescriptor.AVERAGE_LINEAGE_DURATION.getDescriptor()).longValue());
|
||||
Assert.assertEquals(Double.valueOf(68000000/67).longValue(), snapshot.getStatusMetric(ProcessorStatusDescriptor.AVERAGE_TASK_NANOS.getDescriptor()).longValue());
|
||||
}
|
||||
|
||||
protected void assertProcessorWithCounterStatusSnapshot(final StatusSnapshot snapshot, final boolean withCounter) {
|
||||
Assert.assertEquals(85L, snapshot.getStatusMetric(ProcessorStatusDescriptor.BYTES_READ.getDescriptor()).longValue());
|
||||
Assert.assertEquals(86L, snapshot.getStatusMetric(ProcessorStatusDescriptor.BYTES_WRITTEN.getDescriptor()).longValue());
|
||||
Assert.assertEquals(85L+86L, snapshot.getStatusMetric(ProcessorStatusDescriptor.BYTES_TRANSFERRED.getDescriptor()).longValue());
|
||||
Assert.assertEquals(82L, snapshot.getStatusMetric(ProcessorStatusDescriptor.INPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(81L, snapshot.getStatusMetric(ProcessorStatusDescriptor.INPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(84L, snapshot.getStatusMetric(ProcessorStatusDescriptor.OUTPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(83L, snapshot.getStatusMetric(ProcessorStatusDescriptor.OUTPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(87L, snapshot.getStatusMetric(ProcessorStatusDescriptor.TASK_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(88L, snapshot.getStatusMetric(ProcessorStatusDescriptor.TASK_MILLIS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(88000000L, snapshot.getStatusMetric(ProcessorStatusDescriptor.TASK_NANOS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(89L, snapshot.getStatusMetric(ProcessorStatusDescriptor.FLOWFILES_REMOVED.getDescriptor()).longValue());
|
||||
Assert.assertEquals(90L, snapshot.getStatusMetric(ProcessorStatusDescriptor.AVERAGE_LINEAGE_DURATION.getDescriptor()).longValue());
|
||||
Assert.assertEquals(Double.valueOf(88000000/87).longValue(), snapshot.getStatusMetric(ProcessorStatusDescriptor.AVERAGE_TASK_NANOS.getDescriptor()).longValue());
|
||||
|
||||
final Map<String, ? extends MetricDescriptor<?>> counterMetrics = snapshot.getMetricDescriptors()
|
||||
.stream().filter(d -> d instanceof CounterMetricDescriptor).collect(Collectors.toMap(d -> d.getLabel(), d -> d));
|
||||
|
||||
if (withCounter) {
|
||||
Assert.assertEquals(2, counterMetrics.size());
|
||||
Assert.assertEquals(97L, snapshot.getStatusMetric(counterMetrics.get("counter1 (5 mins)")).longValue());
|
||||
Assert.assertEquals(98L, snapshot.getStatusMetric(counterMetrics.get("counter2 (5 mins)")).longValue());
|
||||
} else {
|
||||
Assert.assertTrue(counterMetrics.isEmpty());
|
||||
}
|
||||
}
|
||||
|
||||
protected void assertConnectionStatusSnapshot(final StatusSnapshot snapshot) {
|
||||
Assert.assertEquals(102L, snapshot.getStatusMetric(ConnectionStatusDescriptor.INPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(101L, snapshot.getStatusMetric(ConnectionStatusDescriptor.INPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(106L, snapshot.getStatusMetric(ConnectionStatusDescriptor.OUTPUT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(105L, snapshot.getStatusMetric(ConnectionStatusDescriptor.OUTPUT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(104L, snapshot.getStatusMetric(ConnectionStatusDescriptor.QUEUED_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(103L, snapshot.getStatusMetric(ConnectionStatusDescriptor.QUEUED_COUNT.getDescriptor()).longValue());
|
||||
}
|
||||
|
||||
protected void assertRemoteProcessGroupSnapshot(final StatusSnapshot snapshot) {
|
||||
Assert.assertEquals(123000L, snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.SENT_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(122L, snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.SENT_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(125000L, snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.RECEIVED_BYTES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(124L, snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.RECEIVED_COUNT.getDescriptor()).longValue());
|
||||
Assert.assertEquals(Double.valueOf(125000L/300).longValue(), snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.RECEIVED_BYTES_PER_SECOND.getDescriptor()).longValue());
|
||||
Assert.assertEquals(Double.valueOf(123000L/300).longValue(), snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.SENT_BYTES_PER_SECOND.getDescriptor()).longValue());
|
||||
Assert.assertEquals(Double.valueOf((123000L+125000L)/300).longValue(), snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.TOTAL_BYTES_PER_SECOND.getDescriptor()).longValue());
|
||||
Assert.assertEquals(Double.valueOf(128000L).longValue(), snapshot.getStatusMetric(RemoteProcessGroupStatusDescriptor.AVERAGE_LINEAGE_DURATION.getDescriptor()).longValue());
|
||||
}
|
||||
|
||||
protected NodeStatus givenNodeStatus(final int number) {
|
||||
final NodeStatus result = new NodeStatus();
|
||||
result.setCreatedAtInMs(System.currentTimeMillis());
|
||||
result.setFreeHeap(1 + number);
|
||||
result.setUsedHeap(2 + number);
|
||||
result.setHeapUtilization(3 + number);
|
||||
result.setFreeNonHeap(4 + number);
|
||||
result.setUsedNonHeap(5 + number);
|
||||
result.setOpenFileHandlers(6 + number);
|
||||
result.setProcessorLoadAverage(7.1d + number);
|
||||
result.setTotalThreads(9 + number);
|
||||
result.setEventDrivenThreads(20 + number);
|
||||
result.setTimerDrivenThreads(21 + number);
|
||||
result.setFlowFileRepositoryFreeSpace(10 + number);
|
||||
result.setFlowFileRepositoryUsedSpace(11 + number);
|
||||
result.setContentRepositories(Arrays.asList(
|
||||
givenStorageStatus("c1", 12 + number, 13 + number),
|
||||
givenStorageStatus("c2", 14 + number, 15 + number)
|
||||
));
|
||||
result.setProvenanceRepositories(Arrays.asList(
|
||||
givenStorageStatus("p1", 16 + number, 17 + number),
|
||||
givenStorageStatus("p2", 18 + number, 19 + number)
|
||||
));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
protected NodeStatus givenNodeStatus() {
|
||||
final NodeStatus status = new NodeStatus();
|
||||
status.setFreeHeap(11);
|
||||
status.setUsedHeap(12);
|
||||
status.setHeapUtilization(13);
|
||||
status.setFreeNonHeap(14);
|
||||
status.setUsedNonHeap(15);
|
||||
status.setOpenFileHandlers(16);
|
||||
status.setProcessorLoadAverage(17);
|
||||
status.setTotalThreads(18);
|
||||
status.setEventDrivenThreads(19);
|
||||
status.setTimerDrivenThreads(20);
|
||||
status.setContentRepositories(Arrays.asList(
|
||||
givenStorageStatus("c1", 21, 22),
|
||||
givenStorageStatus("c2", 23, 24)
|
||||
));
|
||||
status.setProvenanceRepositories(Arrays.asList(
|
||||
givenStorageStatus("p1", 25, 26),
|
||||
givenStorageStatus("p2", 27, 28)
|
||||
));
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
protected StorageStatus givenStorageStatus(final String name, final long usedSpace, final long freeSpace) {
|
||||
final StorageStatus storageStatus = new StorageStatus();
|
||||
storageStatus.setName(name);
|
||||
storageStatus.setUsedSpace(usedSpace);
|
||||
storageStatus.setFreeSpace(freeSpace);
|
||||
return storageStatus;
|
||||
}
|
||||
|
||||
protected List<GarbageCollectionStatus> givenGarbageCollectionStatuses(final Date measuredAt, long gc1Count, long gc1Millis, long gc2Count, long gc2Millis) {
|
||||
final GarbageCollectionStatus status1 = new StandardGarbageCollectionStatus("gc1", measuredAt, gc1Count, gc1Millis);
|
||||
final GarbageCollectionStatus status2 = new StandardGarbageCollectionStatus("gc2", measuredAt, gc2Count, gc2Millis);
|
||||
return Arrays.asList(status1, status2);
|
||||
}
|
||||
|
||||
protected List<GarbageCollectionStatus> givenGarbageCollectionStatuses(final Date measuredAt) {
|
||||
return givenGarbageCollectionStatuses(measuredAt, 31, 32, 41, 42);
|
||||
}
|
||||
|
||||
protected void assertNodeStatusHistory(final StatusSnapshot snapshot) {
|
||||
// Default metrics
|
||||
Assert.assertEquals(11, snapshot.getStatusMetric(NodeStatusDescriptor.FREE_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(12, snapshot.getStatusMetric(NodeStatusDescriptor.USED_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(13, snapshot.getStatusMetric(NodeStatusDescriptor.HEAP_UTILIZATION.getDescriptor()).longValue());
|
||||
Assert.assertEquals(14, snapshot.getStatusMetric(NodeStatusDescriptor.FREE_NON_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(15, snapshot.getStatusMetric(NodeStatusDescriptor.USED_NON_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(16, snapshot.getStatusMetric(NodeStatusDescriptor.OPEN_FILE_HANDLES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(17000000, snapshot.getStatusMetric(NodeStatusDescriptor.PROCESSOR_LOAD_AVERAGE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(18, snapshot.getStatusMetric(NodeStatusDescriptor.TOTAL_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(19, snapshot.getStatusMetric(NodeStatusDescriptor.EVENT_DRIVEN_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(20, snapshot.getStatusMetric(NodeStatusDescriptor.TIME_DRIVEN_THREADS.getDescriptor()).longValue());
|
||||
|
||||
// Storage metrics
|
||||
Assert.assertEquals(21, snapshot.getStatusMetric(getDescriptor(snapshot, "contentStorage1Used")).longValue());
|
||||
Assert.assertEquals(22, snapshot.getStatusMetric(getDescriptor(snapshot, "contentStorage1Free")).longValue());
|
||||
Assert.assertEquals(23, snapshot.getStatusMetric(getDescriptor(snapshot, "contentStorage2Used")).longValue());
|
||||
Assert.assertEquals(24, snapshot.getStatusMetric(getDescriptor(snapshot, "contentStorage2Free")).longValue());
|
||||
Assert.assertEquals(25, snapshot.getStatusMetric(getDescriptor(snapshot, "provenanceStorage3Used")).longValue());
|
||||
Assert.assertEquals(26, snapshot.getStatusMetric(getDescriptor(snapshot, "provenanceStorage3Free")).longValue());
|
||||
Assert.assertEquals(27, snapshot.getStatusMetric(getDescriptor(snapshot, "provenanceStorage4Used")).longValue());
|
||||
Assert.assertEquals(28, snapshot.getStatusMetric(getDescriptor(snapshot, "provenanceStorage4Free")).longValue());
|
||||
}
|
||||
|
||||
private MetricDescriptor<?> getDescriptor(final StatusSnapshot snapshot, final String field) {
|
||||
return snapshot.getMetricDescriptors().stream().filter(md -> md.getField().equals(field)).collect(Collectors.toList()).get(0);
|
||||
}
|
||||
|
||||
protected void assertGc1Status(final List<GarbageCollectionStatus> gc1) {
|
||||
Assert.assertEquals(1, gc1.size());
|
||||
final GarbageCollectionStatus status = gc1.get(0);
|
||||
Assert.assertEquals(31, status.getCollectionCount());
|
||||
Assert.assertEquals(32, status.getCollectionMillis());
|
||||
}
|
||||
|
||||
protected void assertGc2Status(final List<GarbageCollectionStatus> gc2) {
|
||||
Assert.assertEquals(1, gc2.size());
|
||||
final GarbageCollectionStatus status = gc2.get(0);
|
||||
Assert.assertEquals(41, status.getCollectionCount());
|
||||
Assert.assertEquals(42, status.getCollectionMillis());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,207 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import io.questdb.MessageBusImpl;
|
||||
import io.questdb.cairo.CairoConfiguration;
|
||||
import io.questdb.cairo.CairoEngine;
|
||||
import io.questdb.cairo.DefaultCairoConfiguration;
|
||||
import io.questdb.cairo.TableWriter;
|
||||
import io.questdb.cairo.sql.Record;
|
||||
import io.questdb.cairo.sql.RecordCursor;
|
||||
import io.questdb.cairo.sql.RecordCursorFactory;
|
||||
import io.questdb.griffin.SqlExecutionContext;
|
||||
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
|
||||
import org.apache.nifi.util.FileUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.time.Instant;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class EmbeddedQuestDbRolloverHandlerTest {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedQuestDbRolloverHandlerTest.class);
|
||||
|
||||
private static final String PATH_BASE = "target/questdb";
|
||||
private String CREATE_TABLE = "CREATE TABLE measurements (capturedAt TIMESTAMP, value INT) TIMESTAMP(capturedAt) PARTITION BY DAY";
|
||||
|
||||
final Instant now = Instant.now();
|
||||
|
||||
private String path;
|
||||
private QuestDbContext dbContext;
|
||||
private EmbeddedQuestDbRolloverHandler testSubject;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
path = PATH_BASE + System.currentTimeMillis();
|
||||
FileUtils.ensureDirectoryExistAndCanReadAndWrite(new File(path));
|
||||
|
||||
dbContext = givenDbContext();
|
||||
testSubject = new EmbeddedQuestDbRolloverHandler(Collections.singletonList("measurements"), 2, dbContext);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
try {
|
||||
FileUtils.deleteFile(new File(path), true);
|
||||
} catch (final Exception e) {
|
||||
LOGGER.error("Could not delete database directory", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollOverWhenWithEmptyDatabase() throws Exception {
|
||||
// given
|
||||
givenTableIsCreated(dbContext);
|
||||
|
||||
// when
|
||||
whenRollOverIsExecuted();
|
||||
|
||||
// then
|
||||
thenRemainingPartitionsAre(Arrays.asList());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollOverWhenLessPartitionThanNeeded() throws Exception {
|
||||
// given
|
||||
givenTableIsCreated(dbContext);
|
||||
givenTableIsPopulated(givenMeasurementTimes(Arrays.asList(0, 1)));
|
||||
|
||||
// when
|
||||
whenRollOverIsExecuted();
|
||||
|
||||
// then
|
||||
thenRemainingPartitionsAre(Arrays.asList(0, 1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollOverWhenNoPartitionToDrop() throws Exception {
|
||||
// given
|
||||
givenTableIsCreated(dbContext);
|
||||
givenTableIsPopulated(givenMeasurementTimes(Arrays.asList(0, 1, 2)));
|
||||
|
||||
// when
|
||||
whenRollOverIsExecuted();
|
||||
|
||||
// then
|
||||
thenRemainingPartitionsAre(Arrays.asList(0, 1, 2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollOverWhenOldPartitionsPresent() throws Exception {
|
||||
// given
|
||||
givenTableIsCreated(dbContext);
|
||||
givenTableIsPopulated(givenMeasurementTimes(Arrays.asList(0, 1, 2, 3, 4)));
|
||||
|
||||
// when
|
||||
whenRollOverIsExecuted();
|
||||
|
||||
// then
|
||||
thenRemainingPartitionsAre(Arrays.asList(0, 1, 2));
|
||||
}
|
||||
|
||||
@Test
|
||||
// This scenario might occurs when the NiFi was stopped and the persistens storage remaing
|
||||
public void testRollOverWhenNonconsecutivePartitionsPresent() throws Exception {
|
||||
// given
|
||||
givenTableIsCreated(dbContext);
|
||||
givenTableIsPopulated(givenMeasurementTimes(Arrays.asList(0, 1, 7, 8, 9)));
|
||||
|
||||
// when
|
||||
whenRollOverIsExecuted();
|
||||
|
||||
// then
|
||||
thenRemainingPartitionsAre(Arrays.asList(0, 1));
|
||||
}
|
||||
|
||||
private QuestDbContext givenDbContext() {
|
||||
final CairoConfiguration configuration = new DefaultCairoConfiguration(path);
|
||||
final CairoEngine engine = new CairoEngine(configuration);
|
||||
return new QuestDbContext(engine, new MessageBusImpl());
|
||||
}
|
||||
|
||||
private void givenTableIsCreated(final QuestDbContext dbContext) throws Exception {
|
||||
dbContext.getCompiler().compile(CREATE_TABLE, dbContext.getSqlExecutionContext());
|
||||
}
|
||||
|
||||
private void givenTableIsPopulated(final List<Long> givenMeasurementTimes) {
|
||||
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
|
||||
final TableWriter tableWriter = dbContext.getEngine().getWriter(executionContext.getCairoSecurityContext(), "measurements");
|
||||
|
||||
for (int i = 0; i < givenMeasurementTimes.size(); i++) {
|
||||
final TableWriter.Row row = tableWriter.newRow(TimeUnit.MILLISECONDS.toMicros(givenMeasurementTimes.get(i)));
|
||||
row.putTimestamp(0, TimeUnit.MILLISECONDS.toMicros(givenMeasurementTimes.get(i)));
|
||||
row.putInt(1, i);
|
||||
row.append();
|
||||
}
|
||||
|
||||
tableWriter.commit();
|
||||
tableWriter.close();
|
||||
}
|
||||
|
||||
private List<Long> givenMeasurementTimes(final List<Integer> daysBack) {
|
||||
final List<Long> result = new LinkedList<>();
|
||||
|
||||
for (final Integer day : daysBack) {
|
||||
result.add(now.minus(day, ChronoUnit.DAYS).toEpochMilli());
|
||||
}
|
||||
|
||||
result.sort((l1, l2) -> l1.compareTo(l2));
|
||||
return result;
|
||||
}
|
||||
|
||||
private void whenRollOverIsExecuted() {
|
||||
testSubject.run();
|
||||
}
|
||||
|
||||
private void thenRemainingPartitionsAre(final List<Integer> expectedDays) throws Exception {
|
||||
final List<String> expectedPartitions = new ArrayList<>(expectedDays.size());
|
||||
|
||||
for (final Integer expectedDay : expectedDays) {
|
||||
expectedPartitions.add(EmbeddedQuestDbRolloverHandler.DATE_FORMATTER.format(now.minus(expectedDay, ChronoUnit.DAYS)));
|
||||
}
|
||||
|
||||
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
|
||||
final RecordCursorFactory cursorFactory = dbContext.getCompiler()
|
||||
.compile(String.format(EmbeddedQuestDbRolloverHandler.SELECTION_QUERY, "measurements"), executionContext).getRecordCursorFactory();
|
||||
final RecordCursor cursor = cursorFactory.getCursor(executionContext);
|
||||
|
||||
final List<String> existingPartitions = new LinkedList<>();
|
||||
|
||||
while (cursor.hasNext()) {
|
||||
final Record record = cursor.getRecord();
|
||||
existingPartitions.add(new StringBuilder(record.getStr(0)).toString());
|
||||
}
|
||||
|
||||
Assert.assertEquals(expectedPartitions.size(), existingPartitions.size());
|
||||
|
||||
for (final String expectedPartition : expectedPartitions) {
|
||||
Assert.assertTrue("Partition " + expectedPartition + " is expected", existingPartitions.contains(expectedPartition));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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.history;
|
||||
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class EmbeddedQuestDbStatusHistoryRepositoryForComponentsTest extends AbstractEmbeddedQuestDbStatusHistoryRepositoryTest {
|
||||
|
||||
@Test
|
||||
public void testReadingEmptyRepository() throws Exception {
|
||||
// when
|
||||
final StatusHistory result = testSubject.getProcessGroupStatusHistory(ROOT_GROUP_ID, START, END, PREFERRED_DATA_POINTS);
|
||||
|
||||
// then
|
||||
assertStatusHistoryIsEmpty(result);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWritingThenReadingComponents() throws Exception {
|
||||
// given
|
||||
testSubject.capture(new NodeStatus(), givenRootProcessGroupStatus(), new ArrayList<>(), INSERTED_AT);
|
||||
givenWaitUntilPersisted();
|
||||
|
||||
// when & then - reading root processor group
|
||||
final StatusHistory rootGroupStatus = testSubject.getProcessGroupStatusHistory(ROOT_GROUP_ID, START, END, PREFERRED_DATA_POINTS);
|
||||
assertCorrectStatusHistory(rootGroupStatus, ROOT_GROUP_ID, "Root");
|
||||
assertRootProcessGroupStatusSnapshot(rootGroupStatus.getStatusSnapshots().get(0));
|
||||
|
||||
// when & then - reading child processor group
|
||||
final StatusHistory childGroupStatus = testSubject.getProcessGroupStatusHistory(CHILD_GROUP_ID, START, END, PREFERRED_DATA_POINTS);
|
||||
assertCorrectStatusHistory(childGroupStatus, CHILD_GROUP_ID, "Child");
|
||||
assertChildProcessGroupStatusSnapshot(childGroupStatus.getStatusSnapshots().get(0));
|
||||
|
||||
// when & then - reading processor (no-counter processor)
|
||||
final StatusHistory processorStatus = testSubject.getProcessorStatusHistory(PROCESSOR_ID, START, END, PREFERRED_DATA_POINTS, false);
|
||||
assertCorrectStatusHistory(processorStatus, PROCESSOR_ID, "Processor");
|
||||
assertProcessorStatusSnapshot(processorStatus.getStatusSnapshots().get(0));
|
||||
|
||||
// when & then - reading processor (with-counter processor)
|
||||
final StatusHistory processorWithCounterStatus1 = testSubject.getProcessorStatusHistory(PROCESSOR_WITH_COUNTER_ID, START, END, PREFERRED_DATA_POINTS, false);
|
||||
assertCorrectStatusHistory(processorWithCounterStatus1, PROCESSOR_WITH_COUNTER_ID, "ProcessorWithCounter");
|
||||
assertProcessorWithCounterStatusSnapshot(processorWithCounterStatus1.getStatusSnapshots().get(0), false);
|
||||
|
||||
// when & then - reading processor (with-counter processor)
|
||||
final StatusHistory processorWithCounterStatus2 = testSubject.getProcessorStatusHistory(PROCESSOR_WITH_COUNTER_ID, START, END, PREFERRED_DATA_POINTS, true);
|
||||
assertCorrectStatusHistory(processorWithCounterStatus2, PROCESSOR_WITH_COUNTER_ID, "ProcessorWithCounter");
|
||||
assertProcessorWithCounterStatusSnapshot(processorWithCounterStatus2.getStatusSnapshots().get(0), true);
|
||||
|
||||
// when & then - reading connection
|
||||
final StatusHistory connectionStatus = testSubject.getConnectionStatusHistory(CONNECTION_ID, START, END, PREFERRED_DATA_POINTS);
|
||||
assertCorrectStatusHistory(connectionStatus, CONNECTION_ID, "Connection");
|
||||
assertConnectionStatusSnapshot(connectionStatus.getStatusSnapshots().get(0));
|
||||
|
||||
// when & then - reading remote process group
|
||||
final StatusHistory remoteProcessGroupStatus = testSubject.getRemoteProcessGroupStatusHistory(REMOTE_PROCESS_GROUP_ID, START, END, PREFERRED_DATA_POINTS);
|
||||
assertCorrectStatusHistory(remoteProcessGroupStatus, REMOTE_PROCESS_GROUP_ID, "RemoteProcessGroup");
|
||||
assertRemoteProcessGroupSnapshot(remoteProcessGroupStatus.getStatusSnapshots().get(0));
|
||||
|
||||
// when & then - requesting data from out of recorded range
|
||||
final StatusHistory rootGroupStatus2 = testSubject.getProcessGroupStatusHistory(ROOT_GROUP_ID, START, END_EARLY, PREFERRED_DATA_POINTS);
|
||||
assertStatusHistoryIsEmpty(rootGroupStatus2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadingLimitedByPreferredDataPoints() throws Exception {
|
||||
// given
|
||||
testSubject.capture(new NodeStatus(), givenSimpleRootProcessGroupStatus(), new ArrayList<>(), new Date(INSERTED_AT.getTime() - TimeUnit.MINUTES.toMillis(8)));
|
||||
testSubject.capture(new NodeStatus(), givenSimpleRootProcessGroupStatus(), new ArrayList<>(), new Date(INSERTED_AT.getTime() - TimeUnit.MINUTES.toMillis(7)));
|
||||
testSubject.capture(new NodeStatus(), givenSimpleRootProcessGroupStatus(), new ArrayList<>(), new Date(INSERTED_AT.getTime() - TimeUnit.MINUTES.toMillis(6)));
|
||||
testSubject.capture(new NodeStatus(), givenSimpleRootProcessGroupStatus(), new ArrayList<>(), new Date(INSERTED_AT.getTime() - TimeUnit.MINUTES.toMillis(5)));
|
||||
givenWaitUntilPersisted();
|
||||
|
||||
// when
|
||||
final StatusHistory result = testSubject.getProcessGroupStatusHistory(ROOT_GROUP_ID, START, END, 3);
|
||||
|
||||
// then - in case the value of preferred data points are lower than the number of snapshots available, the latest will added to the result
|
||||
Assert.assertEquals(3, result.getStatusSnapshots().size());
|
||||
Assert.assertEquals(new Date(INSERTED_AT.getTime() - TimeUnit.MINUTES.toMillis(7)), result.getStatusSnapshots().get(0).getTimestamp());
|
||||
Assert.assertEquals(new Date(INSERTED_AT.getTime() - TimeUnit.MINUTES.toMillis(6)), result.getStatusSnapshots().get(1).getTimestamp());
|
||||
Assert.assertEquals(new Date(INSERTED_AT.getTime() - TimeUnit.MINUTES.toMillis(5)), result.getStatusSnapshots().get(2).getTimestamp());
|
||||
}
|
||||
|
||||
private void assertCorrectStatusHistory(final StatusHistory rootGroupStatus, final String id, final String name) {
|
||||
Assert.assertEquals(id, rootGroupStatus.getComponentDetails().get("Id"));
|
||||
Assert.assertEquals(name, rootGroupStatus.getComponentDetails().get("Name"));
|
||||
Assert.assertEquals(1, rootGroupStatus.getStatusSnapshots().size());
|
||||
}
|
||||
}
|
|
@ -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.history;
|
||||
|
||||
import org.apache.nifi.controller.status.ProcessGroupStatus;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class EmbeddedQuestDbStatusHistoryRepositoryForNodeTest extends AbstractEmbeddedQuestDbStatusHistoryRepositoryTest {
|
||||
|
||||
@Test
|
||||
public void testReadingEmptyRepository() throws Exception {
|
||||
// when
|
||||
final StatusHistory nodeStatusHistory = testSubject.getNodeStatusHistory(START, END);
|
||||
final GarbageCollectionHistory garbageCollectionHistory = testSubject.getGarbageCollectionHistory(START, END);
|
||||
|
||||
// then
|
||||
Assert.assertTrue(nodeStatusHistory.getStatusSnapshots().isEmpty());
|
||||
Assert.assertTrue(garbageCollectionHistory.getGarbageCollectionStatuses("gc1").isEmpty());
|
||||
Assert.assertTrue(garbageCollectionHistory.getGarbageCollectionStatuses("gc2").isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWritingThenReadingComponents() throws Exception {
|
||||
// given
|
||||
testSubject.capture(givenNodeStatus(), new ProcessGroupStatus(), givenGarbageCollectionStatuses(INSERTED_AT), INSERTED_AT);
|
||||
givenWaitUntilPersisted();
|
||||
|
||||
// when & then - reading node status
|
||||
final StatusHistory nodeStatusHistory = testSubject.getNodeStatusHistory(START, END);
|
||||
assertNodeStatusHistory(nodeStatusHistory.getStatusSnapshots().get(0));
|
||||
|
||||
// when & then - garbage collection status
|
||||
final GarbageCollectionHistory garbageCollectionHistory = testSubject.getGarbageCollectionHistory(START, END);
|
||||
assertGc1Status(garbageCollectionHistory.getGarbageCollectionStatuses("gc1"));
|
||||
assertGc2Status(garbageCollectionHistory.getGarbageCollectionStatuses("gc2"));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,223 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
import static org.testng.AssertJUnit.assertEquals;
|
||||
|
||||
/**
|
||||
* This class verifies the VolatileComponentStatusRepository getConnectionStatusHistory method
|
||||
* honors the start/end/preferredDataPoints variables by testing the filterDates function.
|
||||
*/
|
||||
public class VolatileComponentStatusRepositoryForComponentsTest {
|
||||
|
||||
private static VolatileComponentStatusRepository filledRepo;
|
||||
private static VolatileComponentStatusRepository partiallyFilledRepo;
|
||||
private static VolatileComponentStatusRepository emptyRepo;
|
||||
private static final int FIVE_MINUTES = 300000;
|
||||
private static int BUFSIZE3 = 10;
|
||||
|
||||
@BeforeClass
|
||||
public static void createBuffers() {
|
||||
// Fill the repo1 buffer completely with Date objects at five-minute intervals
|
||||
// This provides dates up to around Jul 1979
|
||||
filledRepo = initRepo(1_000_000, 0);
|
||||
|
||||
// Verify partially filled buffers work as expected.
|
||||
partiallyFilledRepo = initRepo(1000, 10);
|
||||
|
||||
emptyRepo = createRepo(BUFSIZE3);
|
||||
}
|
||||
|
||||
private static VolatileComponentStatusRepository initRepo(int bufferSize, int offset) {
|
||||
VolatileComponentStatusRepository repo = createRepo(bufferSize);
|
||||
|
||||
for (long i = 0; i < bufferSize - offset; i++) {
|
||||
repo.timestamps.add(new Date(i * FIVE_MINUTES));
|
||||
}
|
||||
assertEquals(bufferSize - offset, repo.timestamps.getSize());
|
||||
|
||||
return repo;
|
||||
}
|
||||
|
||||
private static VolatileComponentStatusRepository createRepo(int bufferSize) {
|
||||
final NiFiProperties niFiProperties = Mockito.mock(NiFiProperties.class);
|
||||
Mockito.when(niFiProperties.getIntegerProperty(VolatileComponentStatusRepository.NUM_DATA_POINTS_PROPERTY, VolatileComponentStatusRepository.DEFAULT_NUM_DATA_POINTS)).thenReturn(bufferSize);
|
||||
return new VolatileComponentStatusRepository(niFiProperties);
|
||||
}
|
||||
|
||||
private static Date asDate(LocalDateTime localDateTime) {
|
||||
return Date.from(localDateTime.toInstant(ZoneOffset.UTC));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesReturnAll() {
|
||||
testFilterDatesReturnAll(filledRepo);
|
||||
testFilterDatesReturnAll(partiallyFilledRepo);
|
||||
}
|
||||
|
||||
private void testFilterDatesReturnAll(VolatileComponentStatusRepository repo) {
|
||||
List<Date> dates = repo.filterDates(null, null, Integer.MAX_VALUE);
|
||||
assert repo.timestamps != null;
|
||||
assertEquals(repo.timestamps.getSize(), dates.size());
|
||||
assertTrue(dates.equals(repo.timestamps.asList()));
|
||||
repo.timestamps.add(new Date());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingPreferredDataPoints() {
|
||||
List<Date> dates = filledRepo.filterDates(null, null, 1);
|
||||
assertEquals(1, dates.size());
|
||||
assertEquals(filledRepo.timestamps.getNewestElement(), dates.get(0));
|
||||
|
||||
testFilterDatesUsingPreferredDataPoints(filledRepo, 14);
|
||||
testFilterDatesUsingPreferredDataPoints(partiallyFilledRepo, 22);
|
||||
}
|
||||
|
||||
private void testFilterDatesUsingPreferredDataPoints(VolatileComponentStatusRepository repo, int numPoints) {
|
||||
List<Date> dates = repo.filterDates(null, null, numPoints);
|
||||
assertEquals(numPoints, dates.size());
|
||||
assertEquals(repo.timestamps.getNewestElement(), dates.get(dates.size() - 1));
|
||||
assertEquals(repo.timestamps.asList().get(repo.timestamps.getSize() - numPoints), dates.get(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingStartFilter() {
|
||||
// Filter with date that exactly matches an entry in timestamps buffer
|
||||
Date start = asDate(LocalDateTime.of(1978, 1, 1, 0, 45, 0));
|
||||
List<Date> dates = filledRepo.filterDates(start, null, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(filledRepo.timestamps.getNewestElement(), dates.get(dates.size()-1));
|
||||
|
||||
// filter using a date that does not exactly match the time, i.e., not on a five-minute mark
|
||||
start = asDate(LocalDateTime.of(1974, 1, 1, 3, 2, 0));
|
||||
dates = filledRepo.filterDates(start, null, Integer.MAX_VALUE);
|
||||
assertTrue(start.getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(0).getTime() < (start.getTime() + FIVE_MINUTES));
|
||||
assertEquals(filledRepo.timestamps.getNewestElement(), dates.get(dates.size()-1));
|
||||
|
||||
start = asDate(LocalDateTime.of(1970, 1, 1, 0, 0, 0));
|
||||
dates = partiallyFilledRepo.filterDates(start, null, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(partiallyFilledRepo.timestamps.getNewestElement(), dates.get(dates.size()-1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingEndFilter() {
|
||||
// Filter with date that exactly matches an entry in timestamps buffer
|
||||
Date end = asDate(LocalDateTime.of(1970, 2, 1,1, 10, 0));
|
||||
List<Date> dates = filledRepo.filterDates(null, end, Integer.MAX_VALUE);
|
||||
assertEquals(end, dates.get(dates.size()-1));
|
||||
assertEquals(filledRepo.timestamps.getOldestElement(), dates.get(0));
|
||||
|
||||
// filter using a date that does not exactly match the times in buffer
|
||||
end = asDate(LocalDateTime.of(1970, 2, 1,1, 7, 0));
|
||||
dates = filledRepo.filterDates(null, end, Integer.MAX_VALUE);
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size()-1).getTime());
|
||||
assertEquals(dates.get(0), filledRepo.timestamps.getOldestElement());
|
||||
|
||||
end = asDate(LocalDateTime.of(1970, 1, 2,1, 7, 0));
|
||||
dates = partiallyFilledRepo.filterDates(null, end, Integer.MAX_VALUE);
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size()-1).getTime());
|
||||
assertEquals(partiallyFilledRepo.timestamps.asList().get(0), dates.get(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingStartAndEndFilter() {
|
||||
// Filter with dates that exactly matches entries in timestamps buffer
|
||||
Date start = asDate(LocalDateTime.of(1975, 3, 1, 3, 15, 0));
|
||||
Date end = asDate(LocalDateTime.of(1978, 4, 2,4, 25, 0));
|
||||
List<Date> dates = filledRepo.filterDates(start, end, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(end, dates.get(dates.size()-1));
|
||||
|
||||
// Filter with dates that do not exactly matches entries in timestamps buffer
|
||||
start = asDate(LocalDateTime.of(1975, 3, 1, 3, 3, 0));
|
||||
end = asDate(LocalDateTime.of(1977, 4, 2,4, 8, 0));
|
||||
dates = filledRepo.filterDates(start, end, Integer.MAX_VALUE);
|
||||
assertTrue(start.getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(0).getTime() < (start.getTime() + FIVE_MINUTES));
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size()-1).getTime());
|
||||
|
||||
start = asDate(LocalDateTime.of(1970, 1, 1, 3, 15, 0));
|
||||
end = asDate(LocalDateTime.of(1970, 1, 2,4, 25, 0));
|
||||
dates = filledRepo.filterDates(start, end, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(end, dates.get(dates.size()-1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingStartEndAndPreferredFilter() {
|
||||
// Filter with dates that exactly matches entries in timestamps buffer
|
||||
int numPoints = 5;
|
||||
Date start = asDate(LocalDateTime.of(1977, 1, 1, 0, 30, 0));
|
||||
Date end = asDate(LocalDateTime.of(1977, 2, 1,1, 0, 0));
|
||||
List<Date> dates = filledRepo.filterDates(start, end, numPoints);
|
||||
assertEquals(numPoints, dates.size());
|
||||
assertEquals(dates.get(dates.size()-1), end);
|
||||
assertEquals(dates.get(dates.size()-numPoints), new Date(end.getTime() - (numPoints-1)*FIVE_MINUTES));
|
||||
|
||||
// Filter with dates that do not exactly matches entries in timestamps buffer
|
||||
start = asDate(LocalDateTime.of(1975, 1, 1, 0, 31, 0));
|
||||
end = asDate(LocalDateTime.of(1978, 2, 1,1, 59, 0));
|
||||
dates = filledRepo.filterDates(start, end, numPoints);
|
||||
assertTrue(dates.get(0).getTime() < new Date(end.getTime() - (numPoints-1)*FIVE_MINUTES).getTime());
|
||||
assertTrue(new Date(end.getTime() - (numPoints * FIVE_MINUTES)).getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size() - 1).getTime());
|
||||
assertEquals(numPoints, dates.size());
|
||||
|
||||
start = asDate(LocalDateTime.of(1970, 1, 1, 0, 31, 0));
|
||||
end = asDate(LocalDateTime.of(1970, 1, 1,1, 59, 0));
|
||||
dates = partiallyFilledRepo.filterDates(start, end, numPoints);
|
||||
assertTrue(dates.get(0).getTime() < new Date(end.getTime() - (numPoints-1)*FIVE_MINUTES).getTime());
|
||||
assertTrue(new Date(end.getTime() - (numPoints * FIVE_MINUTES)).getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size() - 1).getTime());
|
||||
assertEquals(numPoints, dates.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterWorksWithCircularBuffer() {
|
||||
// Fill repo3 with Date objects at five-minute intervals
|
||||
// This repository is used to verify circular actions behave as expected.
|
||||
for (int i = 0; i < BUFSIZE3 + 15; i++) {
|
||||
emptyRepo.timestamps.add(new Date(i * FIVE_MINUTES));
|
||||
List<Date> dates = emptyRepo.filterDates(null, null, Integer.MAX_VALUE);
|
||||
if (i < BUFSIZE3 - 1) {
|
||||
assertEquals(null, emptyRepo.timestamps.getOldestElement());
|
||||
} else {
|
||||
assertEquals(emptyRepo.timestamps.getOldestElement(), dates.get(0));
|
||||
}
|
||||
assertEquals(emptyRepo.timestamps.asList().get(0), dates.get(0));
|
||||
assertEquals(emptyRepo.timestamps.getNewestElement(), dates.get(dates.size() - 1));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,142 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public class VolatileComponentStatusRepositoryForNodeTest extends AbstractStatusHistoryRepositoryTest {
|
||||
|
||||
@Test
|
||||
public void testNodeStatusHistory() {
|
||||
// given
|
||||
final NiFiProperties niFiProperties = Mockito.mock(NiFiProperties.class);
|
||||
Mockito.when(niFiProperties.getIntegerProperty(VolatileComponentStatusRepository.NUM_DATA_POINTS_PROPERTY, VolatileComponentStatusRepository.DEFAULT_NUM_DATA_POINTS)).thenReturn(10);
|
||||
final VolatileComponentStatusRepository testSubject = new VolatileComponentStatusRepository(niFiProperties);
|
||||
final List<NodeStatus> nodeStatuses = Arrays.asList(
|
||||
givenNodeStatus(0),
|
||||
givenNodeStatus(1)
|
||||
);
|
||||
|
||||
final Date capturedAt = new Date();
|
||||
testSubject.capture(nodeStatuses.get(0), givenSimpleRootProcessGroupStatus(), givenGarbageCollectionStatuses(capturedAt, 1, 100, 2, 300), capturedAt);
|
||||
testSubject.capture(nodeStatuses.get(1), givenSimpleRootProcessGroupStatus(), givenGarbageCollectionStatuses(capturedAt, 1, 100, 5, 700), capturedAt);
|
||||
|
||||
// when
|
||||
final StatusHistory result = testSubject.getNodeStatusHistory(new Date(0), new Date());
|
||||
|
||||
// then
|
||||
// checking on snapshots
|
||||
Assert.assertEquals(nodeStatuses.size(), result.getStatusSnapshots().size());;
|
||||
|
||||
// metrics based on NodeStatus
|
||||
for (int i = 0; i < result.getStatusSnapshots().size(); i++) {
|
||||
final StatusSnapshot snapshot = result.getStatusSnapshots().get(i);
|
||||
final NodeStatus nodeStatus = nodeStatuses.get(i);
|
||||
|
||||
Assert.assertEquals(nodeStatus.getFreeHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.FREE_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getUsedHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.USED_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getHeapUtilization(), snapshot.getStatusMetric(NodeStatusDescriptor.HEAP_UTILIZATION.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getFreeNonHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.FREE_NON_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getUsedNonHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.USED_NON_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getOpenFileHandlers(), snapshot.getStatusMetric(NodeStatusDescriptor.OPEN_FILE_HANDLES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
Double.valueOf(nodeStatus.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.PROCESSOR_LOAD_AVERAGE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getTotalThreads(), snapshot.getStatusMetric(NodeStatusDescriptor.TOTAL_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getEventDrivenThreads(), snapshot.getStatusMetric(NodeStatusDescriptor.EVENT_DRIVEN_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getTimerDrivenThreads(), snapshot.getStatusMetric(NodeStatusDescriptor.TIME_DRIVEN_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getFlowFileRepositoryFreeSpace(), snapshot.getStatusMetric(NodeStatusDescriptor.FLOW_FILE_REPOSITORY_FREE_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getFlowFileRepositoryUsedSpace(), snapshot.getStatusMetric(NodeStatusDescriptor.FLOW_FILE_REPOSITORY_USED_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getContentRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.CONTENT_REPOSITORY_FREE_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getContentRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.CONTENT_REPOSITORY_USED_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getProvenanceRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.PROVENANCE_REPOSITORY_FREE_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getProvenanceRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.PROVENANCE_REPOSITORY_USED_SPACE.getDescriptor()).longValue());
|
||||
|
||||
// metrics based on repositories
|
||||
Assert.assertEquals(12 + i, getMetricAtOrdinal(snapshot, 17)); // c1 used
|
||||
Assert.assertEquals(13 + i, getMetricAtOrdinal(snapshot, 16)); // c1 free
|
||||
Assert.assertEquals(14 + i, getMetricAtOrdinal(snapshot, 19)); // c2 used
|
||||
Assert.assertEquals(15 + i, getMetricAtOrdinal(snapshot, 18)); // c2 free
|
||||
|
||||
Assert.assertEquals(16 + i, getMetricAtOrdinal(snapshot, 21)); // p1 used
|
||||
Assert.assertEquals(17 + i, getMetricAtOrdinal(snapshot, 20)); // p1 free
|
||||
Assert.assertEquals(18 + i, getMetricAtOrdinal(snapshot, 23)); // p2 used
|
||||
Assert.assertEquals(19 + i, getMetricAtOrdinal(snapshot, 22)); // p2 free
|
||||
}
|
||||
|
||||
// metrics based on GarbageCollectionStatus (The ordinal numbers are true for setup, in production it might differ)
|
||||
final int g0TimeOrdinal = 24;
|
||||
final int g0CountOrdinal = 25;
|
||||
final int g0TimeDiffOrdinal = 26;
|
||||
final int g0CountDiffOrdinal = 27;
|
||||
final int g1TimeOrdinal = 28;
|
||||
final int g1CountOrdinal = 29;
|
||||
final int g1TimeDiffOrdinal = 30;
|
||||
final int g1CountDiffOrdinal = 31;
|
||||
|
||||
final StatusSnapshot snapshot1 = result.getStatusSnapshots().get(0);
|
||||
final StatusSnapshot snapshot2 = result.getStatusSnapshots().get(1);
|
||||
|
||||
Assert.assertEquals(100L, getMetricAtOrdinal(snapshot1, g0TimeOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g0TimeDiffOrdinal));
|
||||
Assert.assertEquals(1L, getMetricAtOrdinal(snapshot1, g0CountOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g0CountDiffOrdinal));
|
||||
Assert.assertEquals(300L, getMetricAtOrdinal(snapshot1, g1TimeOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g1TimeDiffOrdinal));
|
||||
Assert.assertEquals(2L, getMetricAtOrdinal(snapshot1, g1CountOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g1CountDiffOrdinal));
|
||||
|
||||
Assert.assertEquals(100L, getMetricAtOrdinal(snapshot2, g0TimeOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot2, g0TimeDiffOrdinal));
|
||||
Assert.assertEquals(1L, getMetricAtOrdinal(snapshot2, g0CountOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot2, g0CountDiffOrdinal));
|
||||
Assert.assertEquals(700L, getMetricAtOrdinal(snapshot2, g1TimeOrdinal));
|
||||
Assert.assertEquals(400L, getMetricAtOrdinal(snapshot2, g1TimeDiffOrdinal));
|
||||
Assert.assertEquals(5L, getMetricAtOrdinal(snapshot2, g1CountOrdinal));
|
||||
Assert.assertEquals(3L, getMetricAtOrdinal(snapshot2, g1CountDiffOrdinal));
|
||||
}
|
||||
|
||||
private static long getMetricAtOrdinal(final StatusSnapshot snapshot, final long ordinal) {
|
||||
final Set<MetricDescriptor<?>> metricDescriptors = snapshot.getMetricDescriptors();
|
||||
|
||||
for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {
|
||||
if (metricDescriptor.getMetricIdentifier() == ordinal) {
|
||||
return snapshot.getStatusMetric(metricDescriptor);
|
||||
}
|
||||
}
|
||||
|
||||
Assert.fail();
|
||||
return Long.MIN_VALUE;
|
||||
}
|
||||
}
|
|
@ -1,400 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history;
|
||||
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.apache.nifi.controller.status.ProcessGroupStatus;
|
||||
import org.apache.nifi.controller.status.StorageStatus;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.testng.Assert;
|
||||
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
import static org.apache.nifi.controller.status.history.VolatileComponentStatusRepository.DEFAULT_NUM_DATA_POINTS;
|
||||
import static org.apache.nifi.controller.status.history.VolatileComponentStatusRepository.NUM_DATA_POINTS_PROPERTY;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
import static org.testng.AssertJUnit.assertEquals;
|
||||
|
||||
/**
|
||||
* This class verifies the VolatileComponentStatusRepository getConnectionStatusHistory method
|
||||
* honors the start/end/preferredDataPoints variables by testing the filterDates function.
|
||||
*/
|
||||
public class VolatileComponentStatusRepositoryTest {
|
||||
|
||||
private static VolatileComponentStatusRepository filledRepo;
|
||||
private static VolatileComponentStatusRepository partiallyFilledRepo;
|
||||
private static VolatileComponentStatusRepository emptyRepo;
|
||||
private static final int FIVE_MINUTES = 300000;
|
||||
private static int BUFSIZE3 = 10;
|
||||
|
||||
@BeforeClass
|
||||
public static void createBuffers() {
|
||||
// Fill the repo1 buffer completely with Date objects at five-minute intervals
|
||||
// This provides dates up to around Jul 1979
|
||||
filledRepo = initRepo(1_000_000, 0);
|
||||
|
||||
// Verify partially filled buffers work as expected.
|
||||
partiallyFilledRepo = initRepo(1000, 10);
|
||||
|
||||
emptyRepo = createRepo(BUFSIZE3);
|
||||
}
|
||||
|
||||
private static VolatileComponentStatusRepository initRepo(int bufferSize, int offset) {
|
||||
VolatileComponentStatusRepository repo = createRepo(bufferSize);
|
||||
|
||||
for (long i = 0; i < bufferSize - offset; i++) {
|
||||
repo.timestamps.add(new Date(i * FIVE_MINUTES));
|
||||
}
|
||||
assertEquals(bufferSize - offset, repo.timestamps.getSize());
|
||||
|
||||
return repo;
|
||||
}
|
||||
|
||||
private static VolatileComponentStatusRepository createRepo(int bufferSize) {
|
||||
NiFiProperties props = mock(NiFiProperties.class);
|
||||
|
||||
when(props.getIntegerProperty(NUM_DATA_POINTS_PROPERTY, DEFAULT_NUM_DATA_POINTS)).thenReturn(bufferSize);
|
||||
|
||||
VolatileComponentStatusRepository repo = new VolatileComponentStatusRepository(props);
|
||||
|
||||
return repo;
|
||||
}
|
||||
|
||||
private static Date asDate(LocalDateTime localDateTime) {
|
||||
return Date.from(localDateTime.toInstant(ZoneOffset.UTC));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesReturnAll() {
|
||||
testFilterDatesReturnAll(filledRepo);
|
||||
testFilterDatesReturnAll(partiallyFilledRepo);
|
||||
}
|
||||
|
||||
private void testFilterDatesReturnAll(VolatileComponentStatusRepository repo) {
|
||||
List<Date> dates = repo.filterDates(null, null, Integer.MAX_VALUE);
|
||||
assert repo.timestamps != null;
|
||||
assertEquals(repo.timestamps.getSize(), dates.size());
|
||||
assertTrue(dates.equals(repo.timestamps.asList()));
|
||||
repo.timestamps.add(new Date());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingPreferredDataPoints() {
|
||||
List<Date> dates = filledRepo.filterDates(null, null, 1);
|
||||
assertEquals(1, dates.size());
|
||||
assertEquals(filledRepo.timestamps.getNewestElement(), dates.get(0));
|
||||
|
||||
testFilterDatesUsingPreferredDataPoints(filledRepo, 14);
|
||||
testFilterDatesUsingPreferredDataPoints(partiallyFilledRepo, 22);
|
||||
}
|
||||
|
||||
private void testFilterDatesUsingPreferredDataPoints(VolatileComponentStatusRepository repo, int numPoints) {
|
||||
List<Date> dates = repo.filterDates(null, null, numPoints);
|
||||
assertEquals(numPoints, dates.size());
|
||||
assertEquals(repo.timestamps.getNewestElement(), dates.get(dates.size() - 1));
|
||||
assertEquals(repo.timestamps.asList().get(repo.timestamps.getSize() - numPoints), dates.get(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingStartFilter() {
|
||||
// Filter with date that exactly matches an entry in timestamps buffer
|
||||
Date start = asDate(LocalDateTime.of(1978, 1, 1, 0, 45, 0));
|
||||
List<Date> dates = filledRepo.filterDates(start, null, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(filledRepo.timestamps.getNewestElement(), dates.get(dates.size()-1));
|
||||
|
||||
// filter using a date that does not exactly match the time, i.e., not on a five-minute mark
|
||||
start = asDate(LocalDateTime.of(1974, 1, 1, 3, 2, 0));
|
||||
dates = filledRepo.filterDates(start, null, Integer.MAX_VALUE);
|
||||
assertTrue(start.getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(0).getTime() < (start.getTime() + FIVE_MINUTES));
|
||||
assertEquals(filledRepo.timestamps.getNewestElement(), dates.get(dates.size()-1));
|
||||
|
||||
start = asDate(LocalDateTime.of(1970, 1, 1, 0, 0, 0));
|
||||
dates = partiallyFilledRepo.filterDates(start, null, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(partiallyFilledRepo.timestamps.getNewestElement(), dates.get(dates.size()-1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingEndFilter() {
|
||||
// Filter with date that exactly matches an entry in timestamps buffer
|
||||
Date end = asDate(LocalDateTime.of(1970, 2, 1,1, 10, 0));
|
||||
List<Date> dates = filledRepo.filterDates(null, end, Integer.MAX_VALUE);
|
||||
assertEquals(end, dates.get(dates.size()-1));
|
||||
assertEquals(filledRepo.timestamps.getOldestElement(), dates.get(0));
|
||||
|
||||
// filter using a date that does not exactly match the times in buffer
|
||||
end = asDate(LocalDateTime.of(1970, 2, 1,1, 7, 0));
|
||||
dates = filledRepo.filterDates(null, end, Integer.MAX_VALUE);
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size()-1).getTime());
|
||||
assertEquals(dates.get(0), filledRepo.timestamps.getOldestElement());
|
||||
|
||||
end = asDate(LocalDateTime.of(1970, 1, 2,1, 7, 0));
|
||||
dates = partiallyFilledRepo.filterDates(null, end, Integer.MAX_VALUE);
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size()-1).getTime());
|
||||
assertEquals(partiallyFilledRepo.timestamps.asList().get(0), dates.get(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingStartAndEndFilter() {
|
||||
// Filter with dates that exactly matches entries in timestamps buffer
|
||||
Date start = asDate(LocalDateTime.of(1975, 3, 1, 3, 15, 0));
|
||||
Date end = asDate(LocalDateTime.of(1978, 4, 2,4, 25, 0));
|
||||
List<Date> dates = filledRepo.filterDates(start, end, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(end, dates.get(dates.size()-1));
|
||||
|
||||
// Filter with dates that do not exactly matches entries in timestamps buffer
|
||||
start = asDate(LocalDateTime.of(1975, 3, 1, 3, 3, 0));
|
||||
end = asDate(LocalDateTime.of(1977, 4, 2,4, 8, 0));
|
||||
dates = filledRepo.filterDates(start, end, Integer.MAX_VALUE);
|
||||
assertTrue(start.getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(0).getTime() < (start.getTime() + FIVE_MINUTES));
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size()-1).getTime());
|
||||
|
||||
start = asDate(LocalDateTime.of(1970, 1, 1, 3, 15, 0));
|
||||
end = asDate(LocalDateTime.of(1970, 1, 2,4, 25, 0));
|
||||
dates = filledRepo.filterDates(start, end, Integer.MAX_VALUE);
|
||||
assertEquals(start, dates.get(0));
|
||||
assertEquals(end, dates.get(dates.size()-1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterDatesUsingStartEndAndPreferredFilter() {
|
||||
// Filter with dates that exactly matches entries in timestamps buffer
|
||||
int numPoints = 5;
|
||||
Date start = asDate(LocalDateTime.of(1977, 1, 1, 0, 30, 0));
|
||||
Date end = asDate(LocalDateTime.of(1977, 2, 1,1, 0, 0));
|
||||
List<Date> dates = filledRepo.filterDates(start, end, numPoints);
|
||||
assertEquals(numPoints, dates.size());
|
||||
assertEquals(dates.get(dates.size()-1), end);
|
||||
assertEquals(dates.get(dates.size()-numPoints), new Date(end.getTime() - (numPoints-1)*FIVE_MINUTES));
|
||||
|
||||
// Filter with dates that do not exactly matches entries in timestamps buffer
|
||||
start = asDate(LocalDateTime.of(1975, 1, 1, 0, 31, 0));
|
||||
end = asDate(LocalDateTime.of(1978, 2, 1,1, 59, 0));
|
||||
dates = filledRepo.filterDates(start, end, numPoints);
|
||||
assertTrue(dates.get(0).getTime() < new Date(end.getTime() - (numPoints-1)*FIVE_MINUTES).getTime());
|
||||
assertTrue(new Date(end.getTime() - (numPoints * FIVE_MINUTES)).getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size() - 1).getTime());
|
||||
assertEquals(numPoints, dates.size());
|
||||
|
||||
start = asDate(LocalDateTime.of(1970, 1, 1, 0, 31, 0));
|
||||
end = asDate(LocalDateTime.of(1970, 1, 1,1, 59, 0));
|
||||
dates = partiallyFilledRepo.filterDates(start, end, numPoints);
|
||||
assertTrue(dates.get(0).getTime() < new Date(end.getTime() - (numPoints-1)*FIVE_MINUTES).getTime());
|
||||
assertTrue(new Date(end.getTime() - (numPoints * FIVE_MINUTES)).getTime() < dates.get(0).getTime());
|
||||
assertTrue(dates.get(dates.size()-1).getTime() < end.getTime());
|
||||
assertTrue((end.getTime() - FIVE_MINUTES) < dates.get(dates.size() - 1).getTime());
|
||||
assertEquals(numPoints, dates.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFilterWorksWithCircularBuffer() {
|
||||
// Fill repo3 with Date objects at five-minute intervals
|
||||
// This repository is used to verify circular actions behave as expected.
|
||||
for (int i = 0; i < BUFSIZE3 + 15; i++) {
|
||||
emptyRepo.timestamps.add(new Date(i * FIVE_MINUTES));
|
||||
List<Date> dates = emptyRepo.filterDates(null, null, Integer.MAX_VALUE);
|
||||
if (i < BUFSIZE3 - 1) {
|
||||
assertEquals(null, emptyRepo.timestamps.getOldestElement());
|
||||
} else {
|
||||
assertEquals(emptyRepo.timestamps.getOldestElement(), dates.get(0));
|
||||
}
|
||||
assertEquals(emptyRepo.timestamps.asList().get(0), dates.get(0));
|
||||
assertEquals(emptyRepo.timestamps.getNewestElement(), dates.get(dates.size() - 1));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNodeStatusHistory() {
|
||||
// given
|
||||
final VolatileComponentStatusRepository testSubject = createRepo(BUFSIZE3);
|
||||
final List<NodeStatus> nodeStatuses = Arrays.asList(
|
||||
givenNodeStatus(0),
|
||||
givenNodeStatus(1)
|
||||
);
|
||||
|
||||
testSubject.capture(nodeStatuses.get(0), givenProcessGroupStatus(), givenGarbageCollectionStatus(1, 100, 2, 300));
|
||||
testSubject.capture(nodeStatuses.get(1), givenProcessGroupStatus(), givenGarbageCollectionStatus(1, 100, 5, 700));
|
||||
|
||||
// when
|
||||
final StatusHistory result = testSubject.getNodeStatusHistory();
|
||||
|
||||
// then
|
||||
// checking on snapshots
|
||||
Assert.assertEquals(nodeStatuses.size(), result.getStatusSnapshots().size());;
|
||||
|
||||
// metrics based on NodeStatus
|
||||
for (int i = 0; i < result.getStatusSnapshots().size(); i++) {
|
||||
final StatusSnapshot snapshot = result.getStatusSnapshots().get(i);
|
||||
final NodeStatus nodeStatus = nodeStatuses.get(i);
|
||||
|
||||
Assert.assertEquals(nodeStatus.getFreeHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.FREE_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getUsedHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.USED_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getHeapUtilization(), snapshot.getStatusMetric(NodeStatusDescriptor.HEAP_UTILIZATION.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getFreeNonHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.FREE_NON_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getUsedNonHeap(), snapshot.getStatusMetric(NodeStatusDescriptor.USED_NON_HEAP.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getOpenFileHandlers(), snapshot.getStatusMetric(NodeStatusDescriptor.OPEN_FILE_HANDLES.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
Double.valueOf(nodeStatus.getProcessorLoadAverage() * MetricDescriptor.FRACTION_MULTIPLIER).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.PROCESSOR_LOAD_AVERAGE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getTotalThreads(), snapshot.getStatusMetric(NodeStatusDescriptor.TOTAL_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getEventDrivenThreads(), snapshot.getStatusMetric(NodeStatusDescriptor.EVENT_DRIVEN_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getTimerDrivenThreads(), snapshot.getStatusMetric(NodeStatusDescriptor.TIME_DRIVEN_THREADS.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getFlowFileRepositoryFreeSpace(), snapshot.getStatusMetric(NodeStatusDescriptor.FLOW_FILE_REPOSITORY_FREE_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(nodeStatus.getFlowFileRepositoryUsedSpace(), snapshot.getStatusMetric(NodeStatusDescriptor.FLOW_FILE_REPOSITORY_USED_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getContentRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.CONTENT_REPOSITORY_FREE_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getContentRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.CONTENT_REPOSITORY_USED_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getProvenanceRepositories().stream().map(r -> r.getFreeSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.PROVENANCE_REPOSITORY_FREE_SPACE.getDescriptor()).longValue());
|
||||
Assert.assertEquals(
|
||||
nodeStatus.getProvenanceRepositories().stream().map(r -> r.getUsedSpace()).reduce(0L, (a, b) -> a + b).longValue(),
|
||||
snapshot.getStatusMetric(NodeStatusDescriptor.PROVENANCE_REPOSITORY_USED_SPACE.getDescriptor()).longValue());
|
||||
|
||||
// metrics based on repositories
|
||||
Assert.assertEquals(12 + i, getMetricAtOrdinal(snapshot, 16)); // c1 free
|
||||
Assert.assertEquals(13 + i, getMetricAtOrdinal(snapshot, 17)); // c1 used
|
||||
Assert.assertEquals(14 + i, getMetricAtOrdinal(snapshot, 18)); // c2 free
|
||||
Assert.assertEquals(15 + i, getMetricAtOrdinal(snapshot, 19)); // c2 used
|
||||
|
||||
Assert.assertEquals(16 + i, getMetricAtOrdinal(snapshot, 20)); // p1 free
|
||||
Assert.assertEquals(17 + i, getMetricAtOrdinal(snapshot, 21)); // p1 used
|
||||
Assert.assertEquals(18 + i, getMetricAtOrdinal(snapshot, 22)); // p2 free
|
||||
Assert.assertEquals(19 + i, getMetricAtOrdinal(snapshot, 23)); // p2 used
|
||||
}
|
||||
|
||||
// metrics based on GarbageCollectionStatus (The ordinal numbers are true for setup, in production it might differ)
|
||||
final int g0TimeOrdinal = 24;
|
||||
final int g0CountOrdinal = 25;
|
||||
final int g0TimeDiffOrdinal = 26;
|
||||
final int g0CountDiffOrdinal = 27;
|
||||
final int g1TimeOrdinal = 28;
|
||||
final int g1CountOrdinal = 29;
|
||||
final int g1TimeDiffOrdinal = 30;
|
||||
final int g1CountDiffOrdinal = 31;
|
||||
|
||||
final StatusSnapshot snapshot1 = result.getStatusSnapshots().get(0);
|
||||
final StatusSnapshot snapshot2 = result.getStatusSnapshots().get(1);
|
||||
|
||||
Assert.assertEquals(100L, getMetricAtOrdinal(snapshot1, g0TimeOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g0TimeDiffOrdinal));
|
||||
Assert.assertEquals(1L, getMetricAtOrdinal(snapshot1, g0CountOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g0CountDiffOrdinal));
|
||||
Assert.assertEquals(300L, getMetricAtOrdinal(snapshot1, g1TimeOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g1TimeDiffOrdinal));
|
||||
Assert.assertEquals(2L, getMetricAtOrdinal(snapshot1, g1CountOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot1, g1CountDiffOrdinal));
|
||||
|
||||
Assert.assertEquals(100L, getMetricAtOrdinal(snapshot2, g0TimeOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot2, g0TimeDiffOrdinal));
|
||||
Assert.assertEquals(1L, getMetricAtOrdinal(snapshot2, g0CountOrdinal));
|
||||
Assert.assertEquals(0L, getMetricAtOrdinal(snapshot2, g0CountDiffOrdinal));
|
||||
Assert.assertEquals(700L, getMetricAtOrdinal(snapshot2, g1TimeOrdinal));
|
||||
Assert.assertEquals(400L, getMetricAtOrdinal(snapshot2, g1TimeDiffOrdinal));
|
||||
Assert.assertEquals(5L, getMetricAtOrdinal(snapshot2, g1CountOrdinal));
|
||||
Assert.assertEquals(3L, getMetricAtOrdinal(snapshot2, g1CountDiffOrdinal));
|
||||
}
|
||||
|
||||
private long getMetricAtOrdinal(final StatusSnapshot snapshot, final long ordinal) {
|
||||
final Set<MetricDescriptor<?>> metricDescriptors = snapshot.getMetricDescriptors();
|
||||
|
||||
for (final MetricDescriptor<?> metricDescriptor : metricDescriptors) {
|
||||
if (metricDescriptor.getMetricIdentifier() == ordinal) {
|
||||
return snapshot.getStatusMetric(metricDescriptor);
|
||||
}
|
||||
}
|
||||
|
||||
Assert.fail();
|
||||
return Long.MIN_VALUE;
|
||||
}
|
||||
|
||||
private NodeStatus givenNodeStatus(final int number) {
|
||||
final NodeStatus result = new NodeStatus();
|
||||
result.setCreatedAtInMs(System.currentTimeMillis());
|
||||
result.setFreeHeap(1 + number);
|
||||
result.setUsedHeap(2 + number);
|
||||
result.setHeapUtilization(3 + number);
|
||||
result.setFreeNonHeap(4 + number);
|
||||
result.setUsedNonHeap(5 + number);
|
||||
result.setOpenFileHandlers(6 + number);
|
||||
result.setProcessorLoadAverage(7.1d + number);
|
||||
result.setTotalThreads(9 + number);
|
||||
result.setEventDrivenThreads(20 + number);
|
||||
result.setTimerDrivenThreads(21 + number);
|
||||
result.setFlowFileRepositoryFreeSpace(10 + number);
|
||||
result.setFlowFileRepositoryUsedSpace(11 + number);
|
||||
result.setContentRepositories(Arrays.asList(
|
||||
givenStorageStatus("c1", 12 + number, 13 + number),
|
||||
givenStorageStatus("c2", 14 + number, 15 + number)
|
||||
));
|
||||
result.setProvenanceRepositories(Arrays.asList(
|
||||
givenStorageStatus("p1", 16 + number, 17 + number),
|
||||
givenStorageStatus("p2", 18 + number, 19 + number)
|
||||
));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private StorageStatus givenStorageStatus(final String name, final long freeSpace, final long usedSpace) {
|
||||
final StorageStatus result = new StorageStatus();
|
||||
result.setName(name);
|
||||
result.setFreeSpace(freeSpace);
|
||||
result.setUsedSpace(usedSpace);
|
||||
return result;
|
||||
}
|
||||
|
||||
private ProcessGroupStatus givenProcessGroupStatus() {
|
||||
final ProcessGroupStatus result = Mockito.mock(ProcessGroupStatus.class);
|
||||
Mockito.when(result.getId()).thenReturn("rootId");
|
||||
Mockito.when(result.getName()).thenReturn("rootName");
|
||||
Mockito.when(result.getProcessorStatus()).thenReturn(Collections.emptyList());
|
||||
Mockito.when(result.getConnectionStatus()).thenReturn(Collections.emptyList());
|
||||
Mockito.when(result.getRemoteProcessGroupStatus()).thenReturn(Collections.emptyList());
|
||||
Mockito.when(result.getProcessGroupStatus()).thenReturn(Collections.emptyList());
|
||||
return result;
|
||||
}
|
||||
|
||||
private List<GarbageCollectionStatus> givenGarbageCollectionStatus(long gc1Count, long gc1Millis, long gc2Count, long gc2Millis) {
|
||||
final List<GarbageCollectionStatus> result = new ArrayList<>(2);
|
||||
result.add(new StandardGarbageCollectionStatus("gc0", new Date(), gc1Count, gc1Millis));
|
||||
result.add(new StandardGarbageCollectionStatus("gc1", new Date(), gc2Count, gc2Millis));
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.status.history.storage;
|
||||
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.apache.nifi.controller.status.NodeStatus;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.junit.MockitoJUnitRunner;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
|
||||
@RunWith(MockitoJUnitRunner.class)
|
||||
public class BufferedWriterForStatusStorageTest {
|
||||
private static final int BUFFER_SIZE = 3;
|
||||
|
||||
@Mock
|
||||
StatusStorage<NodeStatus> payload;
|
||||
|
||||
@Test
|
||||
public void testStoringOnlyWhenPersist() {
|
||||
// given
|
||||
final BufferedWriterForStatusStorage<NodeStatus> testSubject = new BufferedWriterForStatusStorage<>(payload, BUFFER_SIZE);
|
||||
final ArgumentCaptor<List> statusEntriesCaptor = ArgumentCaptor.forClass(List.class);
|
||||
Mockito.doNothing().when(payload).store(statusEntriesCaptor.capture());
|
||||
|
||||
// when
|
||||
for (int i = 0; i <= 5; i++) {
|
||||
testSubject.collect(new ImmutablePair<>(Instant.now(), new NodeStatus()));
|
||||
}
|
||||
|
||||
// then
|
||||
Mockito.verify(payload, Mockito.never()).store(Mockito.anyList());
|
||||
|
||||
// when
|
||||
testSubject.flush();
|
||||
|
||||
// then
|
||||
Mockito.verify(payload, Mockito.only()).store(Mockito.anyList());
|
||||
Assert.assertEquals(BUFFER_SIZE, statusEntriesCaptor.getValue().size());
|
||||
}
|
||||
}
|
|
@ -64,7 +64,7 @@ import org.apache.nifi.controller.serialization.FlowSynchronizer;
|
|||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||
import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
|
||||
import org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider;
|
||||
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
import org.apache.nifi.controller.status.history.VolatileComponentStatusRepository;
|
||||
import org.apache.nifi.encrypt.StringEncryptor;
|
||||
import org.apache.nifi.engine.FlowEngine;
|
||||
|
@ -216,7 +216,7 @@ public class FrameworkIntegrationTest {
|
|||
extensionManager.injectExtensionType(ContentRepository.class, FileSystemRepository.class);
|
||||
extensionManager.injectExtensionType(ProvenanceRepository.class, WriteAheadProvenanceRepository.class);
|
||||
extensionManager.injectExtensionType(StateProvider.class, WriteAheadLocalStateProvider.class);
|
||||
extensionManager.injectExtensionType(ComponentStatusRepository.class, VolatileComponentStatusRepository.class);
|
||||
extensionManager.injectExtensionType(StatusHistoryRepository.class, VolatileComponentStatusRepository.class);
|
||||
extensionManager.injectExtensionType(FlowFileSwapManager.class, FileSystemSwapManager.class);
|
||||
|
||||
extensionManager.injectExtensionType(Processor.class, BiConsumerProcessor.class);
|
||||
|
|
|
@ -27,7 +27,7 @@ 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.controller.status.history.StatusHistoryRepository;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
|
@ -70,7 +70,7 @@ public class NarThreadContextClassLoader extends URLClassLoader {
|
|||
narSpecificClasses.add(AccessPolicyProvider.class);
|
||||
narSpecificClasses.add(LoginIdentityProvider.class);
|
||||
narSpecificClasses.add(ProvenanceRepository.class);
|
||||
narSpecificClasses.add(ComponentStatusRepository.class);
|
||||
narSpecificClasses.add(StatusHistoryRepository.class);
|
||||
narSpecificClasses.add(FlowFileRepository.class);
|
||||
narSpecificClasses.add(FlowFileSwapManager.class);
|
||||
narSpecificClasses.add(ContentRepository.class);
|
||||
|
|
|
@ -16,6 +16,33 @@
|
|||
*/
|
||||
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.analytics.StatusAnalyticsModel;
|
||||
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
|
||||
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.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
|
@ -33,33 +60,6 @@ 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).
|
||||
*
|
||||
|
@ -92,7 +92,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
|
|||
definitionMap.put(AccessPolicyProvider.class, new HashSet<>());
|
||||
definitionMap.put(LoginIdentityProvider.class, new HashSet<>());
|
||||
definitionMap.put(ProvenanceRepository.class, new HashSet<>());
|
||||
definitionMap.put(ComponentStatusRepository.class, new HashSet<>());
|
||||
definitionMap.put(StatusHistoryRepository.class, new HashSet<>());
|
||||
definitionMap.put(FlowFileRepository.class, new HashSet<>());
|
||||
definitionMap.put(FlowFileSwapManager.class, new HashSet<>());
|
||||
definitionMap.put(ContentRepository.class, new HashSet<>());
|
||||
|
|
|
@ -125,6 +125,11 @@
|
|||
<nifi.components.status.repository.buffer.size>1440</nifi.components.status.repository.buffer.size>
|
||||
<nifi.components.status.snapshot.frequency>1 min</nifi.components.status.snapshot.frequency>
|
||||
|
||||
<!-- QuestDb status repository properties -->
|
||||
<nifi.status.repository.questdb.persist.node.days>14</nifi.status.repository.questdb.persist.node.days>
|
||||
<nifi.status.repository.questdb.persist.component.days>3</nifi.status.repository.questdb.persist.component.days>
|
||||
<nifi.status.repository.questdb.persist.location>./status_repository</nifi.status.repository.questdb.persist.location>
|
||||
|
||||
<!-- nifi.properties: web properties -->
|
||||
<nifi.web.http.host>127.0.0.1</nifi.web.http.host>
|
||||
<nifi.web.http.port>8080</nifi.web.http.port>
|
||||
|
|
|
@ -120,11 +120,18 @@ nifi.provenance.repository.concurrent.merge.threads=${nifi.provenance.repository
|
|||
# Volatile Provenance Respository Properties
|
||||
nifi.provenance.repository.buffer.size=${nifi.provenance.repository.buffer.size}
|
||||
|
||||
# Component Status Repository
|
||||
# Component and Node Status History Repository
|
||||
nifi.components.status.repository.implementation=${nifi.components.status.repository.implementation}
|
||||
|
||||
# Volatile Status History Repository Properties
|
||||
nifi.components.status.repository.buffer.size=${nifi.components.status.repository.buffer.size}
|
||||
nifi.components.status.snapshot.frequency=${nifi.components.status.snapshot.frequency}
|
||||
|
||||
# QuestDB Status History Repository Properties
|
||||
nifi.status.repository.questdb.persist.node.days=${nifi.status.repository.questdb.persist.node.days}
|
||||
nifi.status.repository.questdb.persist.component.days=${nifi.status.repository.questdb.persist.component.days}
|
||||
nifi.status.repository.questdb.persist.location=${nifi.status.repository.questdb.persist.location}
|
||||
|
||||
# Site to Site properties
|
||||
nifi.remote.input.host=
|
||||
nifi.remote.input.secure=false
|
||||
|
|
|
@ -59,7 +59,7 @@ 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.controller.status.history.StatusHistoryRepository;
|
||||
import org.apache.nifi.diagnostics.SystemDiagnostics;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
|
@ -290,7 +290,7 @@ public class ControllerFacade implements Authorizable {
|
|||
final StatusHistoryDTO statusHistory = flowController.getNodeStatusHistory();
|
||||
|
||||
if (!authorized) {
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_TYPE, "Node");
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_TYPE, "Node");
|
||||
}
|
||||
|
||||
return statusHistory;
|
||||
|
@ -317,8 +317,8 @@ public class ControllerFacade implements Authorizable {
|
|||
|
||||
// if not authorized
|
||||
if (!authorized) {
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, processorId);
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_TYPE, "Processor");
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_NAME, processorId);
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_TYPE, "Processor");
|
||||
}
|
||||
|
||||
return statusHistory;
|
||||
|
@ -343,9 +343,9 @@ public class ControllerFacade implements Authorizable {
|
|||
|
||||
// if not authorized
|
||||
if (!connection.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser())) {
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, connectionId);
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_SOURCE_NAME, connection.getSource().getIdentifier());
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_DESTINATION_NAME, connection.getDestination().getIdentifier());
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_NAME, connectionId);
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_SOURCE_NAME, connection.getSource().getIdentifier());
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_DESTINATION_NAME, connection.getDestination().getIdentifier());
|
||||
}
|
||||
|
||||
return statusHistory;
|
||||
|
@ -373,7 +373,7 @@ public class ControllerFacade implements Authorizable {
|
|||
|
||||
// if not authorized
|
||||
if (!group.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser())) {
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, groupId);
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_NAME, groupId);
|
||||
}
|
||||
|
||||
return statusHistory;
|
||||
|
@ -398,8 +398,8 @@ public class ControllerFacade implements Authorizable {
|
|||
|
||||
// if not authorized
|
||||
if (!remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser())) {
|
||||
statusHistory.getComponentDetails().put(ComponentStatusRepository.COMPONENT_DETAIL_NAME, remoteProcessGroupId);
|
||||
statusHistory.getComponentDetails().remove(ComponentStatusRepository.COMPONENT_DETAIL_URI);
|
||||
statusHistory.getComponentDetails().put(StatusHistoryRepository.COMPONENT_DETAIL_NAME, remoteProcessGroupId);
|
||||
statusHistory.getComponentDetails().remove(StatusHistoryRepository.COMPONENT_DETAIL_URI);
|
||||
}
|
||||
|
||||
return statusHistory;
|
||||
|
|
Loading…
Reference in New Issue