NIFI-8113 Adding persistent status history repository backed by embedded QuestDB

Signed-off-by: Mark Payne <markap14@hotmail.com>
This commit is contained in:
Bence Simon 2021-02-23 15:58:08 +01:00 committed by Mark Payne
parent 82eedea659
commit 7730777d66
64 changed files with 3965 additions and 557 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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() {

View File

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

View File

@ -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]);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.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);
}
}

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.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);
}

View File

@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.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();
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.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();
}
}
}
}
}

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.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";
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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<>();

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,108 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.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());
}
}

View File

@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.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"));
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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<>());

View File

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

View File

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

View File

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