NIFI-12236 Improved Fault Tolerance in QuestDB Status Repository

- Moved QuestDB components to nifi-questdb-bundle

This closes #8152

Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
Bence Simon 2023-12-12 10:48:02 -06:00 committed by exceptionfactory
parent 4dfbe05641
commit a519585b02
No known key found for this signature in database
GPG Key ID: 29B6A52D2AAE8DBA
109 changed files with 5772 additions and 2373 deletions

View File

@ -1165,7 +1165,7 @@ language governing permissions and limitations under the License. -->
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-questdb-nar</artifactId>
<artifactId>nifi-questdb-status-history-nar</artifactId>
<version>2.0.0-SNAPSHOT</version>
<type>nar</type>
</dependency>

View File

@ -139,6 +139,9 @@ public class NiFiProperties extends ApplicationProperties {
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";
public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_LOCATION_BACKUP = "nifi.status.repository.questdb.persist.location.backup";
public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_BATCH_SIZE = "nifi.status.repository.questdb.persist.batchsize";
public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_FREQUENCY = "nifi.status.repository.questdb.persist.frequency";
// security properties
public static final String SECURITY_KEYSTORE = "nifi.security.keystore";
@ -440,6 +443,9 @@ public class NiFiProperties extends ApplicationProperties {
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";
public static final String DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_LOCATION_BACKUP = "./status_repository_backup";
public static final String DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_BATCH_SIZE = "1000";
public static final String DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_FREQUENCY = "5 secs";
public NiFiProperties() {
this(Collections.emptyMap());
@ -1770,10 +1776,27 @@ public class NiFiProperties extends ApplicationProperties {
/**
* Returns the directory where the QuestDB based status repository is expected to work within.
*
* @return Path object pointing to the database's folder.
* @return String pointing to the database's folder.
*/
public Path getQuestDbStatusRepositoryPath() {
return Paths.get(getProperty(STATUS_REPOSITORY_QUESTDB_PERSIST_LOCATION, DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_LOCATION));
public String getQuestDbStatusRepositoryPath() {
return getProperty(STATUS_REPOSITORY_QUESTDB_PERSIST_LOCATION, DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_LOCATION);
}
/**
* Returns the directory where the backup of the QuestDB based status repository is placed when the database is corrupted.
*
* @return String pointing to the backup folder.
*/
public String getQuestDbStatusRepositoryBackupPath() {
return getProperty(STATUS_REPOSITORY_QUESTDB_PERSIST_LOCATION_BACKUP, DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_LOCATION_BACKUP);
}
public int getQuestDbStatusRepositoryPersistBatchSize() {
return Integer.parseInt(getProperty(STATUS_REPOSITORY_QUESTDB_PERSIST_BATCH_SIZE, DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_BATCH_SIZE));
}
public String getQuestDbStatusRepositoryPersistFrequency() {
return getProperty(STATUS_REPOSITORY_QUESTDB_PERSIST_FREQUENCY, DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_FREQUENCY);
}
/**

View File

@ -3512,7 +3512,7 @@ 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 `org.apache.nifi.controller.status.history.VolatileComponentStatusRepository`,
which stores status history in memory. `org.apache.nifi.controller.status.history.EmbeddedQuestDbStatusHistoryRepository` is also supported and stores status history information on disk so that it is
which stores status history in memory. `org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepository` is also supported and stores status history information on disk so that it is
available across restarts and can be stored for much longer periods of time.
|`nifi.components.status.snapshot.frequency`|This value indicates how often to capture a snapshot of the components' status history. The default value is `1 min`.
|====
@ -3535,8 +3535,8 @@ of 576.
==== Persistent repository
If 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.
If the value of the property `nifi.components.status.repository.implementation` is `org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepository`, the
status history data will be stored to the disk in a persistent manner. Data will be kept between restarts. In order to use persistent repository, the QuestDB NAR must be re-built with the `include-questdb` profiles enabled.
|====
|*Property*|*Description*
@ -3544,6 +3544,10 @@ status history data will be stored to the disk in a persistent manner. Data will
is `14`.
|`nifi.status.repository.questdb.persist.component.days`|The number of days the component status data (i.e., stats for each Processor, Connection, etc.) 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`.
|`nifi.status.repository.questdb.persist.location.backup`|The location of the database backup in case the database is being corrupted and recreated. The default value is `./status_repository_backup`.
|`nifi.status.repository.questdb.persist.batchsize`|The QuestDb based status history repository persists the collected status information in batches. The batch size determines the maximum number of persisted status records at a given time. The default value is `1000`.
|`nifi.status.repository.questdb.persist.frequency`|The frequency of persisting collected status records. The default value is `5 secs`.
|====
[[site_to_site_properties]]

View File

@ -1,129 +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 io.questdb.cairo.sql.Record;
import io.questdb.cairo.sql.RecordCursor;
import io.questdb.cairo.sql.RecordCursorFactory;
import io.questdb.griffin.CompiledQuery;
import io.questdb.griffin.SqlCompiler;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.mp.SCSequence;
import io.questdb.mp.TimeoutBlockingWaitStrategy;
import org.apache.nifi.controller.status.history.questdb.QuestDbContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.ZoneOffset;
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
/**
* 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);
private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd").withZone(ZoneOffset.UTC);
// 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 LIST '%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)";
private final Supplier<ZonedDateTime> timeSource;
private final List<String> tables = new ArrayList<>();
private final int daysToKeepData;
private final QuestDbContext dbContext;
EmbeddedQuestDbRolloverHandler(final Supplier<ZonedDateTime> timeSource, final Collection<String> tables, final int daysToKeepData, final QuestDbContext dbContext) {
this.timeSource = timeSource;
this.tables.addAll(tables);
this.daysToKeepData = daysToKeepData;
this.dbContext = dbContext;
}
public EmbeddedQuestDbRolloverHandler(final Collection<String> tables, final int daysToKeepData, final QuestDbContext dbContext) {
this(() -> ZonedDateTime.now(), tables, daysToKeepData, dbContext);
}
@Override
public void run() {
LOGGER.debug("Rollover started for Tables {}", tables);
tables.forEach(tableName -> rolloverTable(tableName));
LOGGER.debug("Rollover completed for Tables {}", tables);
}
private void rolloverTable(final CharSequence tableName) {
try {
final List<String> partitions = getPartitions(tableName);
final String oldestPartitionToKeep = getOldestPartitionToKeep();
// The last partition if exists, it is considered as "active partition" and cannot be deleted.
for (int i = 0; i < partitions.size() - 1; i++) {
final String partition = partitions.get(i);
if (oldestPartitionToKeep.compareTo(partition) > 0) {
deletePartition(tableName, partition);
}
}
} catch (final Exception e) {
LOGGER.error("Rollover failed for table [{}]", tableName, e);
}
}
private void deletePartition(final CharSequence tableName, final String partition) {
try (final SqlCompiler compiler = dbContext.getCompiler()) {
final CompiledQuery compile = compiler.compile(String.format(DELETION_QUERY, tableName, partition), dbContext.getSqlExecutionContext());
compile.execute(new SCSequence(new TimeoutBlockingWaitStrategy(5, TimeUnit.SECONDS)));
} catch (final Exception e) {
LOGGER.error("Dropping partition [{}] of table [{}] failed", partition, tableName, e);
}
}
private List<String> getPartitions(final CharSequence tableName) throws Exception {
final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext();
final List<String> result = new ArrayList<>(daysToKeepData + 1);
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());
}
}
Collections.sort(result);
return result;
}
private String getOldestPartitionToKeep() {
final ZonedDateTime now = timeSource.get();
final ZonedDateTime utc = now.minusDays(daysToKeepData).withZoneSameInstant(ZoneOffset.UTC);
return utc.format(DATE_FORMATTER);
}
}

View File

@ -1,301 +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 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.ArrayList;
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.ScheduledFuture;
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;
private final List<ScheduledFuture<?>> scheduledFutures = new ArrayList<>();
/**
* 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));
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("Repository start initiated");
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
));
final ScheduledFuture<?> nodeRollerFuture = scheduledExecutorService.scheduleWithFixedDelay(nodeRolloverHandler, ROLL_FREQUENCY, ROLL_FREQUENCY, TimeUnit.MILLISECONDS);
scheduledFutures.add(nodeRollerFuture);
final ScheduledFuture<?> componentRolloverFuture = scheduledExecutorService.scheduleWithFixedDelay(componentRolloverHandler, ROLL_FREQUENCY, ROLL_FREQUENCY, TimeUnit.MILLISECONDS);
scheduledFutures.add(componentRolloverFuture);
final ScheduledFuture<?> writerFuture = scheduledExecutorService.scheduleWithFixedDelay(writer, persistFrequency, persistFrequency, TimeUnit.MILLISECONDS);
scheduledFutures.add(writerFuture);
LOGGER.debug("Repository start completed");
}
@Override
public void shutdown() {
LOGGER.debug("Repository shutdown started");
int cancelCompleted = 0;
int cancelFailed = 0;
for (ScheduledFuture<?> scheduledFuture : scheduledFutures) {
final boolean cancelled = scheduledFuture.cancel(true);
if (cancelled) {
cancelCompleted++;
} else {
cancelFailed++;
}
}
LOGGER.debug("Repository shutdown task cancellation status: completed [{}] failed [{}]", cancelCompleted, cancelFailed);
final List<Runnable> tasks = scheduledExecutorService.shutdownNow();
LOGGER.debug("Repository Scheduled Task Service shutdown remaining tasks [{}]", tasks.size());
dbContext.close();
LOGGER.debug("Repository shutdown completed");
}
@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

@ -1,179 +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.questdb;
import io.questdb.cairo.CairoConfiguration;
import io.questdb.cairo.CairoEngine;
import io.questdb.cairo.DefaultCairoConfiguration;
import io.questdb.griffin.SqlCompiler;
import io.questdb.griffin.SqlCompilerImpl;
import io.questdb.griffin.SqlExecutionContext;
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;
import org.apache.nifi.util.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* 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: Database Status [{}]", 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 String absolutePath = persistLocation.toFile().getAbsolutePath();
final CairoConfiguration configuration = new DefaultCairoConfiguration(absolutePath);
try (
final CairoEngine engine = new CairoEngine(configuration)
) {
LOGGER.info("Database connection successful [{}]", absolutePath);
return true;
} catch (Exception e) {
LOGGER.error("Database connection failed [{}]", absolutePath, e);
return false;
}
}
private static void createDatabase(final Path persistLocation) {
LOGGER.debug("Database creation started [{}]", persistLocation);
final CairoConfiguration configuration;
try {
FileUtils.ensureDirectoryExistAndCanReadAndWrite(persistLocation.toFile());
} catch (final Exception e) {
throw new RuntimeException(String.format("Database directory creation failed [%s]", persistLocation), e);
}
configuration = new DefaultCairoConfiguration(persistLocation.toFile().getAbsolutePath());
try (
final CairoEngine engine = new CairoEngine(configuration);
final SqlCompiler compiler = new SqlCompilerImpl(engine)
) {
final SqlExecutionContext context = SqlExecutionContextFactory.getInstance(engine);
// 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 creation completed [{}]", persistLocation);
} catch (final Exception e) {
throw new RuntimeException(String.format("Database creation failed [%s]", persistLocation), e);
}
}
public static Set<String> getNodeTableNames() {
return NODE_TABLES;
}
public static Set<String> getComponentTableNames() {
return COMPONENT_TABLES;
}
}

View File

@ -1,62 +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.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

@ -1,53 +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.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

@ -1,132 +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.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," +
"totalQueuedDuration LONG," +
"maxQueuedDuration LONG," +
"averageQueuedDuration 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

@ -1,77 +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.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.SqlCompilerImpl;
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 SqlCompilerImpl(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

@ -1,44 +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.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

@ -1,81 +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.questdb;
import io.questdb.cairo.CairoEngine;
import io.questdb.cairo.TableToken;
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;
}
final TableToken tableToken = engine.getTableTokenIfExists(tableName);
if (tableToken == null) {
LOGGER.error("Table Token for table [{}] not found", tableName);
} else {
try (
final TableWriter tableWriter = engine.getWriter(tableToken, "adding rows")
) {
addRows(tableWriter, entries);
tableWriter.commit();
} catch (final Exception e) {
LOGGER.error("Add rows [{}] to table [{}] failed", entries.size(), 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

@ -1,50 +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.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

@ -1,56 +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.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

@ -1,41 +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.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

@ -1,48 +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.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

@ -1,52 +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.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

@ -1,111 +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.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

@ -1,61 +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.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());
METRICS.put(8, ConnectionStatusDescriptor.TOTAL_QUEUED_DURATION.getDescriptor());
METRICS.put(9, ConnectionStatusDescriptor.MAX_QUEUED_DURATION.getDescriptor());
METRICS.put(10, ConnectionStatusDescriptor.AVERAGE_QUEUED_DURATION.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

@ -1,80 +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.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

@ -1,140 +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.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.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

@ -1,62 +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.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

@ -1,167 +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.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

@ -1,60 +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.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

@ -1,136 +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.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

@ -1,59 +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.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.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.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

@ -1,62 +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.storage;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.nifi.controller.status.NodeStatus;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import java.time.Instant;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ExtendWith(MockitoExtension.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());
assertEquals(BUFFER_SIZE, statusEntriesCaptor.getValue().size());
}
}

View File

@ -38,7 +38,6 @@
<module>nifi-framework-leader-election-shared</module>
<module>nifi-framework-zookeeper-leader-election</module>
<module>nifi-framework-kubernetes-bundle</module>
<module>nifi-framework-questdb-bundle</module>
<module>nifi-framework-status-history-shared</module>
<module>nifi-framework-nar-utils</module>
<module>nifi-framework-nar-loading-utils</module>

View File

@ -35,19 +35,7 @@
<module>nifi-headless-server-nar</module>
<module>nifi-framework-external-resource-utils</module>
</modules>
<repositories>
<!-- Shibboleth Repository required for OpenSAML -->
<repository>
<id>shibboleth</id>
<url>https://build.shibboleth.net/nexus/content/repositories/releases/</url>
<releases>
<enabled>true</enabled>
</releases>
<snapshots>
<enabled>false</enabled>
</snapshots>
</repository>
</repositories>
<dependencyManagement>
<dependencies>
<dependency>

View File

@ -16,10 +16,10 @@
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-questdb-bundle</artifactId>
<artifactId>nifi-questdb-bundle</artifactId>
<version>2.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-framework-questdb-nar</artifactId>
<artifactId>nifi-questdb-status-history-nar</artifactId>
<packaging>nar</packaging>
<dependencies>
<dependency>
@ -30,7 +30,7 @@
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-questdb-status-history</artifactId>
<artifactId>nifi-questdb-status-history</artifactId>
<version>2.0.0-SNAPSHOT</version>
</dependency>
</dependencies>

View File

@ -1,4 +1,4 @@
nifi-framework-questdb-nar
nifi-questdb-status-history-nar
Copyright 2014-2023 The Apache Software Foundation
This product includes software developed at

View File

@ -17,10 +17,10 @@
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-questdb-bundle</artifactId>
<artifactId>nifi-questdb-bundle</artifactId>
<version>2.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-framework-questdb-status-history</artifactId>
<artifactId>nifi-questdb-status-history</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
@ -52,9 +52,9 @@
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>org.questdb</groupId>
<artifactId>questdb</artifactId>
<version>7.3.7</version>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-questdb</artifactId>
<version>2.0.0-SNAPSHOT</version>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,190 @@
/*
* 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 org.apache.commons.lang3.concurrent.BasicThreadFactory;
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.GarbageCollectionStatus;
import org.apache.nifi.controller.status.history.StatusSnapshot;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
final class BufferedStatusHistoryStorage implements StatusHistoryStorage {
private static final Logger LOGGER = LoggerFactory.getLogger(BufferedStatusHistoryStorage.class);
private final String id = UUID.randomUUID().toString();
private final List<ScheduledFuture<?>> scheduledFutures = new ArrayList<>();
private final ScheduledExecutorService scheduledExecutorService = Executors
.newScheduledThreadPool(1, new BasicThreadFactory.Builder().namingPattern("BufferedStatusHistoryStorage-" + id + "-%d").build());
private final StatusHistoryStorage storage;
private final long persistFrequencyInMs;
private final int persistBatchSize;
private final BlockingQueue<CapturedStatus<NodeStatus>> nodeStatusQueue = new LinkedBlockingQueue<>();
private final BlockingQueue<CapturedStatus<GarbageCollectionStatus>> garbageCollectionStatusQueue = new LinkedBlockingQueue<>();
private final BlockingQueue<CapturedStatus<ProcessGroupStatus>> processGroupStatusQueue = new LinkedBlockingQueue<>();
private final BlockingQueue<CapturedStatus<ConnectionStatus>> connectionStatusQueue = new LinkedBlockingQueue<>();
private final BlockingQueue<CapturedStatus<RemoteProcessGroupStatus>> remoteProcessGroupStatusQueue = new LinkedBlockingQueue<>();
private final BlockingQueue<CapturedStatus<ProcessorStatus>> processorStatusQueue = new LinkedBlockingQueue<>();
public BufferedStatusHistoryStorage(final StatusHistoryStorage storage, final long persistFrequencyInMs, final int persistBatchSize) {
this.storage = storage;
this.persistFrequencyInMs = persistFrequencyInMs;
this.persistBatchSize = persistBatchSize;
}
@Override
public void init() {
storage.init();
final ScheduledFuture<?> future = scheduledExecutorService.scheduleWithFixedDelay(
new BufferedStatusHistoryStorageWorker(), persistFrequencyInMs, persistFrequencyInMs, TimeUnit.MILLISECONDS);
scheduledFutures.add(future);
LOGGER.info("Flushing is initiated");
}
@Override
public void close() {
storage.close();
LOGGER.debug("Flushing shutdown started");
int cancelCompleted = 0;
int cancelFailed = 0;
for (final ScheduledFuture<?> scheduledFuture : scheduledFutures) {
final boolean cancelled = scheduledFuture.cancel(true);
if (cancelled) {
cancelCompleted++;
} else {
cancelFailed++;
}
}
LOGGER.debug("Flushing shutdown task cancellation status: completed [{}] failed [{}]", cancelCompleted, cancelFailed);
final List<Runnable> tasks = scheduledExecutorService.shutdownNow();
LOGGER.debug("Scheduled Task Service shutdown remaining tasks [{}]", tasks.size());
}
@Override
public List<StatusSnapshot> getConnectionSnapshots(final String componentId, final Date start, final Date end) {
return storage.getConnectionSnapshots(componentId, start, end);
}
@Override
public List<StatusSnapshot> getProcessGroupSnapshots(final String componentId, final Date start, final Date end) {
return storage.getProcessGroupSnapshots(componentId, start, end);
}
@Override
public List<StatusSnapshot> getRemoteProcessGroupSnapshots(final String componentId, final Date start, final Date end) {
return storage.getRemoteProcessGroupSnapshots(componentId, start, end);
}
@Override
public List<StatusSnapshot> getProcessorSnapshots(final String componentId, final Date start, final Date end) {
return storage.getProcessorSnapshots(componentId, start, end);
}
@Override
public List<StatusSnapshot> getProcessorSnapshotsWithCounters(final String componentId, final Date start, final Date end) {
return storage.getProcessorSnapshotsWithCounters(componentId, start, end);
}
@Override
public List<GarbageCollectionStatus> getGarbageCollectionSnapshots(final Date start, final Date end) {
return storage.getGarbageCollectionSnapshots(start, end);
}
@Override
public List<StatusSnapshot> getNodeStatusSnapshots(final Date start, final Date end) {
return storage.getNodeStatusSnapshots(start, end);
}
@Override
public void storeNodeStatuses(final Collection<CapturedStatus<NodeStatus>> statuses) {
nodeStatusQueue.addAll(statuses);
}
@Override
public void storeGarbageCollectionStatuses(final Collection<CapturedStatus<GarbageCollectionStatus>> statuses) {
garbageCollectionStatusQueue.addAll(statuses);
}
@Override
public void storeProcessGroupStatuses(final Collection<CapturedStatus<ProcessGroupStatus>> statuses) {
processGroupStatusQueue.addAll(statuses);
}
@Override
public void storeConnectionStatuses(final Collection<CapturedStatus<ConnectionStatus>> statuses) {
connectionStatusQueue.addAll(statuses);
}
@Override
public void storeRemoteProcessorGroupStatuses(final Collection<CapturedStatus<RemoteProcessGroupStatus>> statuses) {
remoteProcessGroupStatusQueue.addAll(statuses);
}
@Override
public void storeProcessorStatuses(final Collection<CapturedStatus<ProcessorStatus>> statuses) {
processorStatusQueue.addAll(statuses);
}
private class BufferedStatusHistoryStorageWorker implements Runnable {
@Override
public void run() {
LOGGER.debug("Start flushing");
flush(nodeStatusQueue, storage::storeNodeStatuses);
flush(garbageCollectionStatusQueue, storage::storeGarbageCollectionStatuses);
flush(processGroupStatusQueue, storage::storeProcessGroupStatuses);
flush(connectionStatusQueue, storage::storeConnectionStatuses);
flush(remoteProcessGroupStatusQueue, storage::storeRemoteProcessorGroupStatuses);
flush(processorStatusQueue, storage::storeProcessorStatuses);
LOGGER.debug("Finish flushing");
}
private <T> void flush(final BlockingQueue<T> source, final Consumer<Collection<T>> target) {
final ArrayList<T> statusEntries = new ArrayList<>(persistBatchSize);
source.drainTo(statusEntries, persistBatchSize);
if (!statusEntries.isEmpty()) {
try {
target.accept(statusEntries);
} catch (final Exception e) {
LOGGER.error("Error during flushing buffered status history information.", e);
}
}
}
}
}

View File

@ -16,36 +16,36 @@
*/
package org.apache.nifi.controller.status.history.questdb;
import io.questdb.cairo.CairoConfiguration;
import io.questdb.cairo.CairoEngine;
import io.questdb.griffin.SqlCompiler;
import io.questdb.griffin.SqlCompilerImpl;
import io.questdb.griffin.SqlExecutionContext;
import java.time.Instant;
import java.util.Objects;
public class QuestDbContext {
private final CairoEngine engine;
final class CapturedStatus<T> {
private final T status;
private final Instant captured;
public QuestDbContext(final CairoEngine engine) {
this.engine = engine;
CapturedStatus(final T status, final Instant captured) {
this.status = status;
this.captured = captured;
}
public CairoEngine getEngine() {
return engine;
T getStatus() {
return status;
}
public CairoConfiguration getConfiguration() {
return engine.getConfiguration();
Instant getCaptured() {
return captured;
}
public SqlExecutionContext getSqlExecutionContext() {
return SqlExecutionContextFactory.getInstance(engine);
@Override
public boolean equals(final Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
final CapturedStatus<?> that = (CapturedStatus<?>) o;
return Objects.equals(status, that.status) && Objects.equals(captured, that.captured);
}
public SqlCompiler getCompiler() {
return new SqlCompilerImpl(engine);
}
public void close() {
engine.close();
@Override
public int hashCode() {
return Objects.hash(status, captured);
}
}

View File

@ -14,14 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.history;
package org.apache.nifi.controller.status.history.questdb;
import org.apache.nifi.controller.status.history.ComponentDetails;
import java.util.Map;
/**
* Stores and returns the details of a given component. Implementations are expected to be thread safe.
*/
public interface ComponentDetailsStorage {
interface ComponentDetailsStorage {
/**
* Returns with the details of a given component if known. A component is know if it was present in the last call

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 org.apache.nifi.controller.status.history.MetricDescriptor;
import org.apache.nifi.questdb.InsertRowContext;
import org.apache.nifi.questdb.InsertRowDataSource;
import java.util.Iterator;
import java.util.Map;
import java.util.function.Function;
final class ComponentStatusDataSource<T> implements InsertRowDataSource {
private final Iterator<CapturedStatus<T>> statuses;
private final Map<Integer, MetricDescriptor<T>> metricDescriptors;
private final Function<T, String> acquireId;
ComponentStatusDataSource(final Iterator<CapturedStatus<T>> statuses, final Map<Integer, MetricDescriptor<T>> metricDescriptors, final Function<T, String> acquireId) {
this.statuses = statuses;
this.metricDescriptors = metricDescriptors;
this.acquireId = acquireId;
}
@Override
public boolean hasNextToInsert() {
return statuses.hasNext();
}
@Override
public void fillRowData(final InsertRowContext context) {
final CapturedStatus<T> status = statuses.next();
context.initializeRow(status.getCaptured());
context.addString(1, acquireId.apply(status.getStatus()));
for (final Map.Entry<Integer, MetricDescriptor<T>> metric : metricDescriptors.entrySet()) {
context.addLong(metric.getKey(), metric.getValue().getValueFunction().getValue(status.getStatus()));
}
}
}

View File

@ -0,0 +1,100 @@
/*
* 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 org.apache.nifi.controller.status.ProcessorStatus;
import org.apache.nifi.questdb.InsertRowContext;
import org.apache.nifi.questdb.InsertRowDataSource;
import java.time.Instant;
import java.util.Collection;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
final class CounterStatisticsDataSource implements InsertRowDataSource {
private final Iterator<CounterStatistic> counterStatistics;
public CounterStatisticsDataSource(final Iterator<CounterStatistic> counterStatistics) {
this.counterStatistics = counterStatistics;
}
@Override
public boolean hasNextToInsert() {
return counterStatistics.hasNext();
}
@Override
public void fillRowData(final InsertRowContext context) {
final CounterStatistic counterStatistic = counterStatistics.next();
context.initializeRow(counterStatistic.getCaptured())
.addString(1, counterStatistic.getComponentId())
.addString(2, counterStatistic.getName())
.addLong(3, counterStatistic.getValue());
}
static InsertRowDataSource getInstance(Collection<CapturedStatus<ProcessorStatus>> processorStatuses) {
final List<CounterStatistic> counterStatistics = new LinkedList<>();
for (final CapturedStatus<ProcessorStatus> processorStatus : processorStatuses) {
if (processorStatus.getStatus().getCounters() != null) {
for (final Map.Entry<String, Long> counter : processorStatus.getStatus().getCounters().entrySet()) {
counterStatistics.add(new CounterStatistic(
processorStatus.getCaptured(),
processorStatus.getStatus().getId(),
counter.getKey(),
counter.getValue()
));
}
}
}
return new CounterStatisticsDataSource(counterStatistics.iterator());
}
private static class CounterStatistic {
private final Instant captured;
private final String componentId;
private final String name;
private final long value;
CounterStatistic(final Instant captured, final String componentId, final String name, final long value) {
this.captured = captured;
this.componentId = componentId;
this.name = name;
this.value = value;
}
public Instant getCaptured() {
return captured;
}
public String getComponentId() {
return componentId;
}
public String getName() {
return name;
}
public long getValue() {
return value;
}
}
}

View File

@ -0,0 +1,74 @@
/*
* 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 org.apache.nifi.controller.status.ProcessorStatus;
import org.apache.nifi.controller.status.history.CounterMetricDescriptor;
import org.apache.nifi.controller.status.history.MetricDescriptor;
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
import org.apache.nifi.controller.status.history.StatusSnapshot;
import org.apache.nifi.questdb.QueryRowContext;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
final class CounterStatisticsResultProcessor implements QueryResultProcessor<List<StatusSnapshot>> {
private static final Logger LOGGER = LoggerFactory.getLogger(CounterStatisticsResultProcessor.class);
private final List<StatusSnapshot> processorSnapshots;
final Map<Long, StatusSnapshot> processorSnapshotsByTime;
CounterStatisticsResultProcessor(final List<StatusSnapshot> processorSnapshots) {
this.processorSnapshots = processorSnapshots;
processorSnapshotsByTime = processorSnapshots.stream().collect(Collectors.toMap(s -> s.getTimestamp().getTime(), s -> s));
}
@Override
public void processRow(final QueryRowContext context) {
final long counterCreatedAt = TimeUnit.MICROSECONDS.toMillis(context.getTimestamp(0));
final String counterName = context.getString(2);
final long counterValue = context.getLong(3);
final StatusSnapshot processorStatusSnapshot = processorSnapshotsByTime.get(counterCreatedAt);
final MetricDescriptor<ProcessorStatus> metricDescriptor = getMetricDescriptor(counterName);
if (processorStatusSnapshot instanceof StandardStatusSnapshot) {
((StandardStatusSnapshot) processorStatusSnapshot).addStatusMetric(metricDescriptor, counterValue);
} else {
LOGGER.warn("The snapshot is not an instance of StandardStatusSnapshot");
}
}
@Override
public List<StatusSnapshot> getResult() {
return processorSnapshots;
}
private static CounterMetricDescriptor<ProcessorStatus> getMetricDescriptor(final String counterName) {
return new CounterMetricDescriptor<>(
counterName,
counterName + " (5 mins)",
counterName + " (5 mins)",
MetricDescriptor.Formatter.COUNT,
s -> s.getCounters() == null ? null : s.getCounters().get(counterName)
);
}
}

View File

@ -0,0 +1,243 @@
/*
* 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 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.ComponentDetails;
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.StandardStatusHistory;
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.questdb.DatabaseManager;
import org.apache.nifi.questdb.embedded.EmbeddedDatabaseManagerBuilder;
import org.apache.nifi.questdb.rollover.RolloverStrategy;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_COMPONENT_COUNTER;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_CONNECTION_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_GARBAGE_COLLECTION_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_NODE_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_PROCESSOR_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_PROCESS_GROUP_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_REMOTE_PROCESS_GROUP_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CREATE_STORAGE_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_COMPONENT_COUNTER;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_CONNECTION_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_GARBAGE_COLLECTION_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_NODE_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_PROCESSOR_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_PROCESS_GROUP_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_STORAGE_STATUS;
public class EmbeddedQuestDbStatusHistoryRepository implements StatusHistoryRepository {
private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedQuestDbStatusHistoryRepository.class);
private final InMemoryComponentDetailsStorage componentDetailsProvider = new InMemoryComponentDetailsStorage();
private final NiFiProperties niFiProperties;
private DatabaseManager databaseManager;
private StatusHistoryStorage storage;
public EmbeddedQuestDbStatusHistoryRepository(final NiFiProperties niFiProperties) {
this.niFiProperties = niFiProperties;
}
@Override
public void start() {
LOGGER.debug("Repository start initiated");
final RolloverStrategy nodeStatusRolloverStrategy = RolloverStrategy.deleteOld(getDaysToKeepNodeData(niFiProperties));
final RolloverStrategy componentStatusRolloverStrategy = RolloverStrategy.deleteOld(getDaysToKeepComponentData(niFiProperties));
databaseManager = EmbeddedDatabaseManagerBuilder
.builder(niFiProperties.getQuestDbStatusRepositoryPath())
.backupLocation(niFiProperties.getQuestDbStatusRepositoryBackupPath())
.numberOfAttemptedRetries(2)
.lockAttemptTime(50, TimeUnit.MILLISECONDS)
.rolloverFrequency(10, TimeUnit.MINUTES)
.addTable(TABLE_NAME_NODE_STATUS, CREATE_NODE_STATUS, nodeStatusRolloverStrategy)
.addTable(TABLE_NAME_STORAGE_STATUS, CREATE_STORAGE_STATUS, nodeStatusRolloverStrategy)
.addTable(TABLE_NAME_GARBAGE_COLLECTION_STATUS, CREATE_GARBAGE_COLLECTION_STATUS, nodeStatusRolloverStrategy)
.addTable(TABLE_NAME_PROCESSOR_STATUS, CREATE_PROCESSOR_STATUS, componentStatusRolloverStrategy)
.addTable(TABLE_NAME_CONNECTION_STATUS, CREATE_CONNECTION_STATUS, componentStatusRolloverStrategy)
.addTable(TABLE_NAME_PROCESS_GROUP_STATUS, CREATE_PROCESS_GROUP_STATUS, componentStatusRolloverStrategy)
.addTable(TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS, CREATE_REMOTE_PROCESS_GROUP_STATUS, componentStatusRolloverStrategy)
.addTable(TABLE_NAME_COMPONENT_COUNTER, CREATE_COMPONENT_COUNTER, componentStatusRolloverStrategy)
.build();
storage = new BufferedStatusHistoryStorage(
new QuestDbStatusHistoryStorage(databaseManager.acquireClient()),
FormatUtils.getTimeDuration(niFiProperties.getQuestDbStatusRepositoryPersistFrequency(), TimeUnit.MILLISECONDS),
niFiProperties.getQuestDbStatusRepositoryPersistBatchSize()
);
storage.init();
LOGGER.debug("Repository start completed");
}
@Override
public void shutdown() {
LOGGER.debug("Repository shutdown started");
databaseManager.close();
storage.close();
LOGGER.debug("Repository shutdown completed");
}
@Override
public void capture(final NodeStatus nodeStatus, final ProcessGroupStatus rootGroupStatus, final List<GarbageCollectionStatus> garbageCollectionStatus, final Date timestamp) {
final Instant captured = timestamp.toInstant();
captureNodeStatus(nodeStatus, captured);
captureGarbageCollectionStatus(garbageCollectionStatus, captured);
captureComponentStatus(rootGroupStatus, captured);
updateComponentDetails(rootGroupStatus);
}
private void captureNodeStatus(final NodeStatus nodeStatus, final Instant captured) {
storage.storeNodeStatuses(Collections.singleton(new CapturedStatus<>(nodeStatus, captured)));
}
private void captureGarbageCollectionStatus(final List<GarbageCollectionStatus> statuses, final Instant captured) {
final Set<CapturedStatus<GarbageCollectionStatus>> capturedStatuses = new HashSet<>(statuses.size());
statuses.forEach(status -> capturedStatuses.add(new CapturedStatus<>(status, captured)));
storage.storeGarbageCollectionStatuses(capturedStatuses);
}
private void captureComponentStatus(final ProcessGroupStatus groupStatus, final Instant captured) {
storage.storeProcessGroupStatuses(Collections.singleton(new CapturedStatus<>(groupStatus, captured)));
storage.storeConnectionStatuses(wrapConnectionStatuses(groupStatus, captured));
storage.storeRemoteProcessorGroupStatuses(wrapRemoteProcessGroupStatuses(groupStatus, captured));
storage.storeProcessorStatuses(wrapProcessorStatuses(groupStatus, captured));
groupStatus.getProcessGroupStatus().forEach(child -> captureComponentStatus(child, captured));
}
private Collection<CapturedStatus<ConnectionStatus>> wrapConnectionStatuses(final ProcessGroupStatus groupStatus, final Instant captured) {
final Collection<ConnectionStatus> statuses = groupStatus.getConnectionStatus();
final Set<CapturedStatus<ConnectionStatus>> result = new HashSet<>(statuses.size());
statuses.forEach(status -> result.add(new CapturedStatus<>(status, captured)));
return result;
}
private Collection<CapturedStatus<RemoteProcessGroupStatus>> wrapRemoteProcessGroupStatuses(final ProcessGroupStatus groupStatus, final Instant captured) {
final Collection<RemoteProcessGroupStatus> statuses = groupStatus.getRemoteProcessGroupStatus();
final Set<CapturedStatus<RemoteProcessGroupStatus>> result = new HashSet<>(statuses.size());
statuses.forEach(status -> result.add(new CapturedStatus<>(status, captured)));
return result;
}
private Collection<CapturedStatus<ProcessorStatus>> wrapProcessorStatuses(final ProcessGroupStatus groupStatus, final Instant captured) {
final Collection<ProcessorStatus> statuses = groupStatus.getProcessorStatus();
final Set<CapturedStatus<ProcessorStatus>> result = new HashSet<>(statuses.size());
statuses.forEach(status -> result.add(new CapturedStatus<>(status, captured)));
return result;
}
@Override
public StatusHistory getConnectionStatusHistory(final String connectionId, final Date start, final Date end, final int preferredDataPoints) {
return generateStatusHistory(connectionId, storage.getConnectionSnapshots(connectionId, start, end), preferredDataPoints);
}
@Override
public StatusHistory getProcessGroupStatusHistory(final String processGroupId, final Date start, final Date end, final int preferredDataPoints) {
return generateStatusHistory(processGroupId, storage.getProcessGroupSnapshots(processGroupId, start, end), preferredDataPoints);
}
@Override
public StatusHistory getProcessorStatusHistory(final String processorId, final Date start, final Date end, final int preferredDataPoints, final boolean includeCounters) {
return includeCounters
? generateStatusHistory(processorId, storage.getProcessorSnapshotsWithCounters(processorId, start, end), preferredDataPoints)
: generateStatusHistory(processorId, storage.getProcessorSnapshots(processorId, start, end), preferredDataPoints);
}
@Override
public StatusHistory getRemoteProcessGroupStatusHistory(final String remoteGroupId, final Date start, final Date end, final int preferredDataPoints) {
return generateStatusHistory(remoteGroupId, storage.getRemoteProcessGroupSnapshots(remoteGroupId, start, end), preferredDataPoints);
}
@Override
public StatusHistory getNodeStatusHistory(final Date start, final Date end) {
return new StandardStatusHistory(storage.getNodeStatusSnapshots(start, end), new HashMap<>(), new Date());
}
@Override
public GarbageCollectionHistory getGarbageCollectionHistory(final Date start, final Date end) {
final List<GarbageCollectionStatus> snapshots = storage.getGarbageCollectionSnapshots(start, end);
final StandardGarbageCollectionHistory result = new StandardGarbageCollectionHistory();
snapshots.forEach(result::addGarbageCollectionStatus);
return result;
}
private StatusHistory generateStatusHistory(final String componentId, final List<StatusSnapshot> snapshots, final int preferredDataPoints) {
return new StandardStatusHistory(
new ArrayList<>(snapshots.subList(Math.max(snapshots.size() - preferredDataPoints, 0), snapshots.size())),
componentDetailsProvider.getDetails(componentId),
new Date()
);
}
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);
}
/**
* 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));
}
}

View File

@ -0,0 +1,353 @@
/*
* 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 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.ConnectionStatusDescriptor;
import org.apache.nifi.controller.status.history.GarbageCollectionStatus;
import org.apache.nifi.controller.status.history.MetricDescriptor;
import org.apache.nifi.controller.status.history.NodeStatusDescriptor;
import org.apache.nifi.controller.status.history.ProcessGroupStatusDescriptor;
import org.apache.nifi.controller.status.history.ProcessorStatusDescriptor;
import org.apache.nifi.controller.status.history.RemoteProcessGroupStatusDescriptor;
import org.apache.nifi.controller.status.history.StandardMetricDescriptor;
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.mapping.RequestMapping;
import org.apache.nifi.questdb.mapping.RequestMappingBuilder;
import java.time.ZoneId;
import java.time.format.DateTimeFormatter;
import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
final class EmbeddedQuestDbStatusHistoryRepositoryDefinitions {
/**
* Date format expected by the storage.
*/
static final String CAPTURE_DATE_FORMAT = "yyyy-MM-dd:HH:mm:ss Z";
/**
* Date formatter for the database fields.
*/
static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern(CAPTURE_DATE_FORMAT).withZone(ZoneId.systemDefault());
// General component
static final String COMPONENT_STATUS_QUERY =
"SELECT * FROM %s " +
"WHERE componentId = '%s' " +
"AND captured > to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"AND captured < to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"ORDER BY captured ASC";
// Connection
static final String TABLE_NAME_CONNECTION_STATUS = "connectionStatus";
static final String CREATE_CONNECTION_STATUS =
"CREATE TABLE " + TABLE_NAME_CONNECTION_STATUS + " (" +
"captured TIMESTAMP," +
"componentId SYMBOL capacity 2000 nocache index capacity 1500," +
"inputBytes LONG," +
"inputCount LONG," +
"outputBytes LONG," +
"outputCount LONG," +
"queuedBytes LONG," +
"queuedCount LONG," +
"totalQueuedDuration LONG," +
"maxQueuedDuration LONG," +
"averageQueuedDuration LONG" +
") TIMESTAMP(captured) PARTITION BY DAY";
private static final Map<Integer, MetricDescriptor<ConnectionStatus>> CONNECTION_METRICS = new HashMap<>();
static {
CONNECTION_METRICS.put(2, ConnectionStatusDescriptor.INPUT_BYTES.getDescriptor());
CONNECTION_METRICS.put(3, ConnectionStatusDescriptor.INPUT_COUNT.getDescriptor());
CONNECTION_METRICS.put(4, ConnectionStatusDescriptor.OUTPUT_BYTES.getDescriptor());
CONNECTION_METRICS.put(5, ConnectionStatusDescriptor.OUTPUT_COUNT.getDescriptor());
CONNECTION_METRICS.put(6, ConnectionStatusDescriptor.QUEUED_BYTES.getDescriptor());
CONNECTION_METRICS.put(7, ConnectionStatusDescriptor.QUEUED_COUNT.getDescriptor());
CONNECTION_METRICS.put(8, ConnectionStatusDescriptor.TOTAL_QUEUED_DURATION.getDescriptor());
CONNECTION_METRICS.put(9, ConnectionStatusDescriptor.MAX_QUEUED_DURATION.getDescriptor());
CONNECTION_METRICS.put(10, ConnectionStatusDescriptor.AVERAGE_QUEUED_DURATION.getDescriptor());
}
static InsertRowDataSource getConnectionStatusDataSource(final Collection<CapturedStatus<ConnectionStatus>> statuses) {
return new ComponentStatusDataSource<>(statuses.iterator(), CONNECTION_METRICS, ConnectionStatus::getId);
}
static final RequestMapping<StandardStatusSnapshot> CONNECTION_STATUS_REQUEST_MAPPING = getSnapshotRequestMapping(ConnectionStatus.class, CONNECTION_METRICS.values());
// Processor
static final String TABLE_NAME_PROCESSOR_STATUS = "processorStatus";
static final String CREATE_PROCESSOR_STATUS =
"CREATE TABLE " + TABLE_NAME_PROCESSOR_STATUS + " (" +
"captured 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(captured) PARTITION BY DAY";
private static final Map<Integer, MetricDescriptor<ProcessorStatus>> PROCESSOR_METRICS = new HashMap<>();
static {
PROCESSOR_METRICS.put(2, ProcessorStatusDescriptor.BYTES_READ.getDescriptor());
PROCESSOR_METRICS.put(3, ProcessorStatusDescriptor.BYTES_WRITTEN.getDescriptor());
PROCESSOR_METRICS.put(4, ProcessorStatusDescriptor.BYTES_TRANSFERRED.getDescriptor());
PROCESSOR_METRICS.put(5, ProcessorStatusDescriptor.INPUT_BYTES.getDescriptor());
PROCESSOR_METRICS.put(6, ProcessorStatusDescriptor.INPUT_COUNT.getDescriptor());
PROCESSOR_METRICS.put(7, ProcessorStatusDescriptor.OUTPUT_BYTES.getDescriptor());
PROCESSOR_METRICS.put(8, ProcessorStatusDescriptor.OUTPUT_COUNT.getDescriptor());
PROCESSOR_METRICS.put(9, ProcessorStatusDescriptor.TASK_COUNT.getDescriptor());
PROCESSOR_METRICS.put(10, ProcessorStatusDescriptor.TASK_MILLIS.getDescriptor());
PROCESSOR_METRICS.put(11, ProcessorStatusDescriptor.TASK_NANOS.getDescriptor());
PROCESSOR_METRICS.put(12, ProcessorStatusDescriptor.FLOWFILES_REMOVED.getDescriptor());
PROCESSOR_METRICS.put(13, ProcessorStatusDescriptor.AVERAGE_LINEAGE_DURATION.getDescriptor());
PROCESSOR_METRICS.put(14, ProcessorStatusDescriptor.AVERAGE_TASK_NANOS.getDescriptor());
}
static InsertRowDataSource getProcessorStatusDataSource(final Collection<CapturedStatus<ProcessorStatus>> statuses) {
return new ComponentStatusDataSource<>(statuses.iterator(), PROCESSOR_METRICS, ProcessorStatus::getId);
}
static InsertRowDataSource getCounterStatisticsDataSource(final Collection<CapturedStatus<ProcessorStatus>> statuses) {
return CounterStatisticsDataSource.getInstance(statuses);
}
static final RequestMapping<StandardStatusSnapshot> PROCESSOR_STATUS_REQUEST_MAPPING = getSnapshotRequestMapping(ProcessorStatus.class, PROCESSOR_METRICS.values());
// Process group
static final String TABLE_NAME_PROCESS_GROUP_STATUS = "processGroupStatus";
static final String CREATE_PROCESS_GROUP_STATUS =
"CREATE TABLE " + TABLE_NAME_PROCESS_GROUP_STATUS + " (" +
"captured 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(captured) PARTITION BY DAY";
private static final Map<Integer, MetricDescriptor<ProcessGroupStatus>> PROCESS_GROUP_METRICS = new HashMap<>();
static {
PROCESS_GROUP_METRICS.put(2, ProcessGroupStatusDescriptor.BYTES_READ.getDescriptor());
PROCESS_GROUP_METRICS.put(3, ProcessGroupStatusDescriptor.BYTES_WRITTEN.getDescriptor());
PROCESS_GROUP_METRICS.put(4, ProcessGroupStatusDescriptor.BYTES_TRANSFERRED.getDescriptor());
PROCESS_GROUP_METRICS.put(5, ProcessGroupStatusDescriptor.INPUT_BYTES.getDescriptor());
PROCESS_GROUP_METRICS.put(6, ProcessGroupStatusDescriptor.INPUT_COUNT.getDescriptor());
PROCESS_GROUP_METRICS.put(7, ProcessGroupStatusDescriptor.OUTPUT_BYTES.getDescriptor());
PROCESS_GROUP_METRICS.put(8, ProcessGroupStatusDescriptor.OUTPUT_COUNT.getDescriptor());
PROCESS_GROUP_METRICS.put(9, ProcessGroupStatusDescriptor.QUEUED_BYTES.getDescriptor());
PROCESS_GROUP_METRICS.put(10, ProcessGroupStatusDescriptor.QUEUED_COUNT.getDescriptor());
PROCESS_GROUP_METRICS.put(11, ProcessGroupStatusDescriptor.TASK_MILLIS.getDescriptor());
}
static InsertRowDataSource getProcessGroupStatusDataSource(final Collection<CapturedStatus<ProcessGroupStatus>> statuses) {
return new ComponentStatusDataSource<>(statuses.iterator(), PROCESS_GROUP_METRICS, ProcessGroupStatus::getId);
}
static final RequestMapping<StandardStatusSnapshot> PROCESS_GROUP_STATUS_REQUEST_MAPPING = getSnapshotRequestMapping(ProcessGroupStatus.class, PROCESS_GROUP_METRICS.values());
// Remote process group
static final String TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS = "remoteProcessGroupStatus";
static final String CREATE_REMOTE_PROCESS_GROUP_STATUS =
"CREATE TABLE " + TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS + " (" +
"captured 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(captured) PARTITION BY DAY";
private static final Map<Integer, MetricDescriptor<RemoteProcessGroupStatus>> REMOTE_PROCESS_GROUP_METRICS = new HashMap<>();
static {
REMOTE_PROCESS_GROUP_METRICS.put(2, RemoteProcessGroupStatusDescriptor.SENT_BYTES.getDescriptor());
REMOTE_PROCESS_GROUP_METRICS.put(3, RemoteProcessGroupStatusDescriptor.SENT_COUNT.getDescriptor());
REMOTE_PROCESS_GROUP_METRICS.put(4, RemoteProcessGroupStatusDescriptor.RECEIVED_BYTES.getDescriptor());
REMOTE_PROCESS_GROUP_METRICS.put(5, RemoteProcessGroupStatusDescriptor.RECEIVED_COUNT.getDescriptor());
REMOTE_PROCESS_GROUP_METRICS.put(6, RemoteProcessGroupStatusDescriptor.RECEIVED_BYTES_PER_SECOND.getDescriptor());
REMOTE_PROCESS_GROUP_METRICS.put(7, RemoteProcessGroupStatusDescriptor.SENT_BYTES_PER_SECOND.getDescriptor());
REMOTE_PROCESS_GROUP_METRICS.put(8, RemoteProcessGroupStatusDescriptor.TOTAL_BYTES_PER_SECOND.getDescriptor());
REMOTE_PROCESS_GROUP_METRICS.put(9, RemoteProcessGroupStatusDescriptor.AVERAGE_LINEAGE_DURATION.getDescriptor());
}
static InsertRowDataSource getRemoteProcessGroupStatusDataSource(final Collection<CapturedStatus<RemoteProcessGroupStatus>> statuses) {
return new ComponentStatusDataSource<>(statuses.iterator(), REMOTE_PROCESS_GROUP_METRICS, RemoteProcessGroupStatus::getId);
}
static final RequestMapping<StandardStatusSnapshot> REMOTE_PROCESS_GROUP_STATUS_REQUEST_MAPPING = getSnapshotRequestMapping(RemoteProcessGroupStatus.class, REMOTE_PROCESS_GROUP_METRICS.values());
// Garbage collection status
static final String TABLE_NAME_GARBAGE_COLLECTION_STATUS = "garbageCollectionStatus";
static final String CREATE_GARBAGE_COLLECTION_STATUS =
"CREATE TABLE " + TABLE_NAME_GARBAGE_COLLECTION_STATUS + " (" +
"captured TIMESTAMP," +
"memoryManagerName SYMBOL capacity 4 nocache," +
"collectionCount LONG," +
"collectionMinis LONG" +
") TIMESTAMP(captured) PARTITION BY DAY";
static final String STATUS_QUERY_GARBAGE_COLLECTION =
"SELECT * FROM garbageCollectionStatus " +
"WHERE captured > to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"AND captured < to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"ORDER BY captured ASC";
static InsertRowDataSource getGarbageCollectionStatusDataSource(final Collection<CapturedStatus<GarbageCollectionStatus>> statuses) {
return new GarbageCollectionStatusDataSource(statuses.iterator());
}
// Component counter
static final String TABLE_NAME_COMPONENT_COUNTER = "componentCounter";
static final String CREATE_COMPONENT_COUNTER =
"CREATE TABLE " + TABLE_NAME_COMPONENT_COUNTER + " (" +
"captured TIMESTAMP," +
"componentId SYMBOL capacity 2000 nocache index capacity 1500," +
"name SYMBOL capacity 256 nocache," +
"value LONG" +
") TIMESTAMP(captured) PARTITION BY DAY";
// Storage status
static final String TABLE_NAME_STORAGE_STATUS = "storageStatus";
static final String CREATE_STORAGE_STATUS =
"CREATE TABLE " + TABLE_NAME_STORAGE_STATUS + " (" +
"captured TIMESTAMP," +
"name SYMBOL capacity 256 nocache," +
"storageType SHORT," +
"freeSpace LONG," +
"usedSpace LONG" +
") TIMESTAMP(captured) PARTITION BY DAY";
static final String STORAGE_STATUS_QUERY =
"SELECT * FROM storageStatus " +
"WHERE captured > to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"AND captured < to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"ORDER BY captured ASC";
static QueryResultProcessor<Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>>> getStorageStatusResultProcessor() {
return new StorageStatusResultProcessor(NODE_STATUS_METRICS);
}
// Node status
static final String TABLE_NAME_NODE_STATUS = "nodeStatus";
static final String CREATE_NODE_STATUS =
"CREATE TABLE " + TABLE_NAME_NODE_STATUS + " (" +
"captured TIMESTAMP," +
"freeHeap LONG," +
"usedHeap LONG," +
"heapUtilization LONG," +
"freeNonHeap LONG," +
"usedNonHeap LONG," +
"openFileHandlers LONG," +
"processorLoadAverage DOUBLE," +
"totalThreads LONG," +
"timerDrivenThreads LONG" +
") TIMESTAMP(captured) PARTITION BY DAY";
static final String NODE_STATUS_QUERY =
"SELECT * FROM nodeStatus " +
"WHERE captured > to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"AND captured < to_timestamp('%s', '" + CAPTURE_DATE_FORMAT + "') " +
"ORDER BY captured ASC";
private static final Map<Integer, MetricDescriptor<NodeStatus>> NODE_STATUS_METRICS = new HashMap<>();
static {
NODE_STATUS_METRICS.put(1, NodeStatusDescriptor.FREE_HEAP.getDescriptor());
NODE_STATUS_METRICS.put(2, NodeStatusDescriptor.USED_HEAP.getDescriptor());
NODE_STATUS_METRICS.put(3, NodeStatusDescriptor.HEAP_UTILIZATION.getDescriptor());
NODE_STATUS_METRICS.put(4, NodeStatusDescriptor.FREE_NON_HEAP.getDescriptor());
NODE_STATUS_METRICS.put(5, NodeStatusDescriptor.USED_NON_HEAP.getDescriptor());
NODE_STATUS_METRICS.put(6, NodeStatusDescriptor.OPEN_FILE_HANDLES.getDescriptor());
NODE_STATUS_METRICS.put(7, NodeStatusDescriptor.PROCESSOR_LOAD_AVERAGE.getDescriptor());
NODE_STATUS_METRICS.put(8, NodeStatusDescriptor.TOTAL_THREADS.getDescriptor());
NODE_STATUS_METRICS.put(9, NodeStatusDescriptor.TIME_DRIVEN_THREADS.getDescriptor());
}
static InsertRowDataSource getNodeStatusDataSource(final Collection<CapturedStatus<NodeStatus>> statuses) {
return new NodeStatusDataSource(statuses.iterator(), NODE_STATUS_METRICS);
}
static QueryResultProcessor<List<StandardStatusSnapshot>> getNodeStatusResultProcessor(
final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> statusMetricsByTime
) {
return new NodeStatusResultProcessor(NODE_STATUS_METRICS, statusMetricsByTime);
}
private static <T> RequestMapping<StandardStatusSnapshot> getSnapshotRequestMapping(Class<T> type, Collection<MetricDescriptor<T>> descriptorSource) {
final RequestMappingBuilder<StandardStatusSnapshot> requestMappingBuilder = RequestMappingBuilder
.of(() -> new StandardStatusSnapshot(new HashSet<>(descriptorSource)))
.addLongField((snapshot, field) -> snapshot.setTimestamp(new Date(TimeUnit.MICROSECONDS.toMillis(field))))
.addStringField((snapshot, field) -> {}); // Id is not used
descriptorSource.forEach(descriptor -> requestMappingBuilder.addLongField((snapshot, field) -> snapshot.addStatusMetric(descriptor, field)));
return requestMappingBuilder.build();
}
private EmbeddedQuestDbStatusHistoryRepositoryDefinitions() {
// Not to be instantiated
}
}

View File

@ -0,0 +1,49 @@
/*
* 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 org.apache.nifi.controller.status.history.GarbageCollectionStatus;
import org.apache.nifi.controller.status.history.StandardGarbageCollectionStatus;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.QueryRowContext;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
/**
* Special handling for garbage collection status is necessary due to its immutable nature: all the state attributes
* are needed at the time of creation.
*/
final class GarbageCollectionResultProcessor implements QueryResultProcessor<List<GarbageCollectionStatus>> {
private final List<GarbageCollectionStatus> result = new ArrayList<>();
@Override
public void processRow(final QueryRowContext context) {
result.add(new StandardGarbageCollectionStatus(
context.getString(1),
new Date(context.getTimestamp(0)),
context.getLong(2),
context.getLong(3)
));
}
@Override
public List<GarbageCollectionStatus> getResult() {
return result;
}
}

View File

@ -0,0 +1,45 @@
/*
* 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 org.apache.nifi.controller.status.history.GarbageCollectionStatus;
import org.apache.nifi.questdb.InsertRowContext;
import org.apache.nifi.questdb.InsertRowDataSource;
import java.util.Iterator;
final class GarbageCollectionStatusDataSource implements InsertRowDataSource {
private final Iterator<CapturedStatus<GarbageCollectionStatus>> statuses;
GarbageCollectionStatusDataSource(final Iterator<CapturedStatus<GarbageCollectionStatus>> statuses) {
this.statuses = statuses;
}
@Override
public boolean hasNextToInsert() {
return statuses.hasNext();
}
@Override
public void fillRowData(final InsertRowContext context) {
final CapturedStatus<GarbageCollectionStatus> status = statuses.next();
context.initializeRow(status.getCaptured())
.addString(1, status.getStatus().getMemoryManagerName())
.addLong(2, status.getStatus().getCollectionCount())
.addLong(3, status.getStatus().getCollectionMillis());
}
}

View File

@ -14,7 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.history;
package org.apache.nifi.controller.status.history.questdb;
import org.apache.nifi.controller.status.history.ComponentDetails;
import java.util.Collections;
import java.util.HashMap;
@ -24,7 +26,7 @@ import java.util.concurrent.atomic.AtomicReference;
/**
* Implementation of {@link ComponentDetailsStorage} using in memory data structure.
*/
public class InMemoryComponentDetailsStorage implements ComponentDetailsStorage {
final class InMemoryComponentDetailsStorage implements ComponentDetailsStorage {
private final AtomicReference<Map<String, ComponentDetails>> componentDetails = new AtomicReference<>(new HashMap<>());
@Override

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.questdb;
import org.apache.nifi.controller.status.NodeStatus;
import org.apache.nifi.controller.status.history.MetricDescriptor;
import org.apache.nifi.questdb.InsertRowContext;
import org.apache.nifi.questdb.InsertRowDataSource;
import java.util.Iterator;
import java.util.Map;
final class NodeStatusDataSource implements InsertRowDataSource {
private final Iterator<CapturedStatus<NodeStatus>> statuses;
private final Map<Integer, MetricDescriptor<NodeStatus>> metrics;
NodeStatusDataSource(final Iterator<CapturedStatus<NodeStatus>> statuses, final Map<Integer, MetricDescriptor<NodeStatus>> metrics) {
this.statuses = statuses;
this.metrics = metrics;
}
@Override
public boolean hasNextToInsert() {
return statuses.hasNext();
}
@Override
public void fillRowData(final InsertRowContext context) {
final CapturedStatus<NodeStatus> status = statuses.next();
context.initializeRow(status.getCaptured());
for (final Map.Entry<Integer, MetricDescriptor<NodeStatus>> metric : metrics.entrySet()) {
context.addLong(metric.getKey(), metric.getValue().getValueFunction().getValue(status.getStatus()));
}
}
}

View File

@ -0,0 +1,70 @@
/*
* 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 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.StandardStatusSnapshot;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.QueryRowContext;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
final class NodeStatusResultProcessor implements QueryResultProcessor<List<StandardStatusSnapshot>> {
private final Map<Integer, MetricDescriptor<NodeStatus>> metricDescriptors;
private final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> storageMetricsByTime;
private final List<StandardStatusSnapshot> result = new ArrayList<>();
NodeStatusResultProcessor(
final Map<Integer, MetricDescriptor<NodeStatus>> metricDescriptors,
final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> storageMetricsByTime
) {
this.metricDescriptors = metricDescriptors;
this.storageMetricsByTime = storageMetricsByTime;
}
@Override
public void processRow(final QueryRowContext context) {
final long createdAt = TimeUnit.MICROSECONDS.toMillis(context.getTimestamp(0));
final Map<StandardMetricDescriptor<NodeStatus>, Long> storageMetrics = storageMetricsByTime.get(createdAt);
final Set<MetricDescriptor<?>> snapshotMetrics = new HashSet<>(metricDescriptors.size() + storageMetrics.keySet().size());
snapshotMetrics.addAll(metricDescriptors.values());
snapshotMetrics.addAll(storageMetrics.keySet());
final StandardStatusSnapshot snapshot = new StandardStatusSnapshot(snapshotMetrics);
snapshot.setTimestamp(new Date(TimeUnit.MICROSECONDS.toMillis(context.getTimestamp(0))));
for (final Map.Entry<Integer, MetricDescriptor<NodeStatus>> metricDescriptor : metricDescriptors.entrySet()) {
snapshot.addStatusMetric(metricDescriptor.getValue(), context.getLong(metricDescriptor.getKey()));
}
storageMetrics.entrySet().forEach(entry -> snapshot.addStatusMetric(entry.getKey(), entry.getValue()));
result.add(snapshot);
}
@Override
public List<StandardStatusSnapshot> getResult() {
return result;
}
}

View File

@ -0,0 +1,181 @@
/*
* 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 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.GarbageCollectionStatus;
import org.apache.nifi.controller.status.history.StandardMetricDescriptor;
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
import org.apache.nifi.controller.status.history.StatusSnapshot;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.mapping.RequestMapping;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.COMPONENT_STATUS_QUERY;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.CONNECTION_STATUS_REQUEST_MAPPING;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.NODE_STATUS_QUERY;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.PROCESSOR_STATUS_REQUEST_MAPPING;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.PROCESS_GROUP_STATUS_REQUEST_MAPPING;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.REMOTE_PROCESS_GROUP_STATUS_REQUEST_MAPPING;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.STATUS_QUERY_GARBAGE_COLLECTION;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.STORAGE_STATUS_QUERY;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_COMPONENT_COUNTER;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_CONNECTION_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_GARBAGE_COLLECTION_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_NODE_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_PROCESSOR_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_PROCESS_GROUP_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS;
import static org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepositoryDefinitions.TABLE_NAME_STORAGE_STATUS;
final class QuestDbStatusHistoryStorage implements StatusHistoryStorage {
private static final Logger LOGGER = LoggerFactory.getLogger(QuestDbStatusHistoryStorage.class);
private final Client client;
QuestDbStatusHistoryStorage(final Client client) {
this.client = client;
}
@Override
public List<StatusSnapshot> getConnectionSnapshots(final String componentId, final Date start, final Date end) {
return getComponentSnapshots(TABLE_NAME_CONNECTION_STATUS, componentId, CONNECTION_STATUS_REQUEST_MAPPING, start, end);
}
@Override
public List<StatusSnapshot> getProcessGroupSnapshots(final String componentId, final Date start, final Date end) {
return getComponentSnapshots(TABLE_NAME_PROCESS_GROUP_STATUS, componentId, PROCESS_GROUP_STATUS_REQUEST_MAPPING, start, end);
}
@Override
public List<StatusSnapshot> getRemoteProcessGroupSnapshots(final String componentId, final Date start, final Date end) {
return getComponentSnapshots(TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS, componentId, REMOTE_PROCESS_GROUP_STATUS_REQUEST_MAPPING, start, end);
}
@Override
public List<StatusSnapshot> getProcessorSnapshots(final String componentId, final Date start, final Date end) {
return getComponentSnapshots(TABLE_NAME_PROCESSOR_STATUS, componentId, PROCESSOR_STATUS_REQUEST_MAPPING, start, end);
}
@Override
public List<StatusSnapshot> getProcessorSnapshotsWithCounters(final String componentId, final Date start, final Date end) {
final List<StatusSnapshot> componentSnapshots = getComponentSnapshots(TABLE_NAME_PROCESSOR_STATUS, componentId, PROCESSOR_STATUS_REQUEST_MAPPING, start, end);
final String query = String.format(COMPONENT_STATUS_QUERY, TABLE_NAME_COMPONENT_COUNTER, componentId, getStartTime(start), getEndTime(end));
return getResult(query, new CounterStatisticsResultProcessor(componentSnapshots), Collections.emptyList());
}
@Override
public List<GarbageCollectionStatus> getGarbageCollectionSnapshots(final Date start, final Date end) {
final String query = String.format(STATUS_QUERY_GARBAGE_COLLECTION, getStartTime(start), getEndTime(end));
return getResult(query, new GarbageCollectionResultProcessor(), Collections.emptyList());
}
@Override
public List<StatusSnapshot> getNodeStatusSnapshots(final Date start, final Date end) {
final String storageStatusQuery = String.format(STORAGE_STATUS_QUERY, getStartTime(start), getEndTime(end));
final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> statusMetricsByTime
= getResult(storageStatusQuery, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getStorageStatusResultProcessor(), new HashMap<>());
final String nodeStatusQuery = String.format(NODE_STATUS_QUERY, getStartTime(start), getEndTime(end));
return getSnapshot(nodeStatusQuery, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getNodeStatusResultProcessor(statusMetricsByTime));
}
@Override
public void storeNodeStatuses(final Collection<CapturedStatus<NodeStatus>> statuses) {
store(TABLE_NAME_NODE_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getNodeStatusDataSource(statuses));
store(TABLE_NAME_STORAGE_STATUS, StorageStatusDataSource.getInstance(statuses));
}
@Override
public void storeGarbageCollectionStatuses(final Collection<CapturedStatus<GarbageCollectionStatus>> statuses) {
store(TABLE_NAME_GARBAGE_COLLECTION_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getGarbageCollectionStatusDataSource(statuses));
}
@Override
public void storeProcessGroupStatuses(final Collection<CapturedStatus<ProcessGroupStatus>> statuses) {
store(TABLE_NAME_PROCESS_GROUP_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getProcessGroupStatusDataSource(statuses));
}
@Override
public void storeConnectionStatuses(final Collection<CapturedStatus<ConnectionStatus>> statuses) {
store(TABLE_NAME_CONNECTION_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getConnectionStatusDataSource(statuses));
}
@Override
public void storeRemoteProcessorGroupStatuses(final Collection<CapturedStatus<RemoteProcessGroupStatus>> statuses) {
store(TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getRemoteProcessGroupStatusDataSource(statuses));
}
@Override
public void storeProcessorStatuses(final Collection<CapturedStatus<ProcessorStatus>> statuses) {
store(TABLE_NAME_PROCESSOR_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getProcessorStatusDataSource(statuses));
store(TABLE_NAME_COMPONENT_COUNTER, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getCounterStatisticsDataSource(statuses));
}
private <T> void store(final String tableName, final InsertRowDataSource source) {
try {
client.insert(tableName, source);
} catch (final DatabaseException e) {
LOGGER.error("Error during storing snapshots to table [{}]", tableName, e);
}
}
private List<StatusSnapshot> getComponentSnapshots(final String tableName, final String componentId, final RequestMapping<StandardStatusSnapshot> mapping, final Date start, final Date end) {
final String query = String.format(COMPONENT_STATUS_QUERY, tableName, componentId, getStartTime(start), getEndTime(end));
return getSnapshot(query, RequestMapping.getResultProcessor(mapping));
}
private List<StatusSnapshot> getSnapshot(final String query, final QueryResultProcessor<List<StandardStatusSnapshot>> rowProcessor) {
return new ArrayList<>(getResult(query, rowProcessor, Collections.emptyList()));
}
private <T> T getResult(final String query, final QueryResultProcessor<T> rowProcessor, final T errorResult) {
try {
return client.query(query, rowProcessor);
} catch (final DatabaseException e) {
LOGGER.error("Error during returning results for query {}", query, e);
return errorResult;
}
}
private static String getStartTime(final Date start) {
final Instant startTime = (start == null) ? Instant.now().minus(1, ChronoUnit.DAYS) : start.toInstant();
return EmbeddedQuestDbStatusHistoryRepositoryDefinitions.DATE_FORMATTER.format(startTime);
}
private static String getEndTime(final Date end) {
final Instant endTime = (end == null) ? Instant.now() : end.toInstant();
return EmbeddedQuestDbStatusHistoryRepositoryDefinitions.DATE_FORMATTER.format(endTime);
}
}

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.questdb;
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.GarbageCollectionStatus;
import org.apache.nifi.controller.status.history.StatusSnapshot;
import java.util.Collection;
import java.util.Date;
import java.util.List;
interface StatusHistoryStorage {
default void init() {};
default void close() {};
List<StatusSnapshot> getConnectionSnapshots(final String componentId, final Date start, final Date end);
List<StatusSnapshot> getProcessGroupSnapshots(final String componentId, final Date start, final Date end);
List<StatusSnapshot> getRemoteProcessGroupSnapshots(final String componentId, final Date start, final Date end);
List<StatusSnapshot> getProcessorSnapshots(final String componentId, final Date start, final Date end);
List<StatusSnapshot> getProcessorSnapshotsWithCounters(final String componentId, final Date start, final Date end);
List<StatusSnapshot> getNodeStatusSnapshots(final Date start, final Date end);
List<GarbageCollectionStatus> getGarbageCollectionSnapshots(final Date start, final Date end);
void storeNodeStatuses(final Collection<CapturedStatus<NodeStatus>> statuses);
void storeGarbageCollectionStatuses(final Collection<CapturedStatus<GarbageCollectionStatus>> statuses);
void storeProcessGroupStatuses(final Collection<CapturedStatus<ProcessGroupStatus>> statuses);
void storeConnectionStatuses(final Collection<CapturedStatus<ConnectionStatus>> statuses);
void storeRemoteProcessorGroupStatuses(final Collection<CapturedStatus<RemoteProcessGroupStatus>> statuses);
void storeProcessorStatuses(final Collection<CapturedStatus<ProcessorStatus>> statuses);
}

View File

@ -14,22 +14,24 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.history.storage;
package org.apache.nifi.controller.status.history.questdb;
/**
* Wraps a writer object in order to buffer incoming store requests and dispatch the incoming store requests in batches.
*/
public interface BufferedEntryWriter<T> {
enum StorageMetric {
FREE("Free", "Free Space"), USED("Used", "Used Space");
/**
* Collects an entity to write into the internal buffer.
*
* @param entryToStore The entry to store.
*/
void collect(T entryToStore);
private final String field;
private final String label;
/**
* Initiates the store of the payload by sending buffered items to store.
*/
void flush();
StorageMetric(final String field, final String label) {
this.field = field;
this.label = label;
}
String getField() {
return field;
}
String getLabel() {
return label;
}
}

View File

@ -0,0 +1,87 @@
/*
* 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 org.apache.nifi.controller.status.NodeStatus;
import org.apache.nifi.controller.status.StorageStatus;
import org.apache.nifi.questdb.InsertRowContext;
import org.apache.nifi.questdb.InsertRowDataSource;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
final class StorageStatusDataSource implements InsertRowDataSource {
private final Iterator<StorageStatusStatistic> statuses;
private StorageStatusDataSource(final Iterator<StorageStatusStatistic> statuses) {
this.statuses = statuses;
}
@Override
public boolean hasNextToInsert() {
return statuses.hasNext();
}
@Override
public void fillRowData(final InsertRowContext context) {
final StorageStatusStatistic status = statuses.next();
context.initializeRow(status.getCaptured());
context.addString(1, status.getStatus().getName());
context.addShort(2, status.getType().getId());
context.addLong(3, status.getStatus().getFreeSpace());
context.addLong(4, status.getStatus().getUsedSpace());
}
static InsertRowDataSource getInstance(final Collection<CapturedStatus<NodeStatus>> nodeStatuses) {
final List<StorageStatusStatistic> statuses = new ArrayList<>();
for (final CapturedStatus<NodeStatus> nodeStatus : nodeStatuses) {
final Instant captured = nodeStatus.getCaptured();
nodeStatus.getStatus().getContentRepositories().forEach(storageStatus -> statuses.add(new StorageStatusStatistic(storageStatus, StorageStatusType.CONTENT, captured)));
nodeStatus.getStatus().getProvenanceRepositories().forEach(storageStatus -> statuses.add(new StorageStatusStatistic(storageStatus, StorageStatusType.PROVENANCE, captured)));
}
return new StorageStatusDataSource(statuses.iterator());
}
private static class StorageStatusStatistic {
private final StorageStatus status;
private final StorageStatusType type;
private final Instant captured;
private StorageStatusStatistic(final StorageStatus status, final StorageStatusType type, final Instant captured) {
this.status = status;
this.type = type;
this.captured = captured;
}
public StorageStatus getStatus() {
return status;
}
public StorageStatusType getType() {
return type;
}
public Instant getCaptured() {
return captured;
}
}
}

View File

@ -0,0 +1,87 @@
/*
* 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 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.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.QueryRowContext;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
final class StorageStatusResultProcessor implements QueryResultProcessor<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 final Map<Integer, MetricDescriptor<NodeStatus>> metricDescriptors;
private final Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> result = new HashMap<>();
private int storageNumber = 1;
StorageStatusResultProcessor(final Map<Integer, MetricDescriptor<NodeStatus>> metricDescriptors) {
this.metricDescriptors = metricDescriptors;
}
@Override
public void processRow(final QueryRowContext context) {
final long createdAt = TimeUnit.MICROSECONDS.toMillis(context.getTimestamp(0));
final String name = context.getString(1);
final short type = context.getShort(2);
if (!result.containsKey(createdAt)) {
result.put(createdAt, new HashMap<>());
}
final StorageStatusType storageStatusType = StorageStatusType.getById(type);
result.get(createdAt).put(getDescriptor(
metricDescriptors.size() + result.get(createdAt).size(),
getField(storageStatusType, storageNumber, StorageMetric.FREE),
getLabel(storageStatusType, name, StorageMetric.FREE),
STORAGE_FREE_DESCRIPTION
), context.getLong(3));
result.get(createdAt).put(getDescriptor(
metricDescriptors.size() + result.get(createdAt).size(),
getField(storageStatusType, storageNumber, StorageMetric.USED),
getLabel(storageStatusType, name, StorageMetric.USED),
STORAGE_USED_DESCRIPTION
), context.getLong(4));
storageNumber++;
}
@Override
public Map<Long, Map<StandardMetricDescriptor<NodeStatus>, Long>> getResult() {
return result;
}
private StandardMetricDescriptor<NodeStatus> getDescriptor(final int ordinal, final String field, final String label, final String description) {
return new StandardMetricDescriptor<>(() -> ordinal, field, label, description, MetricDescriptor.Formatter.DATA_SIZE, v -> 0L);
}
private String getField(final StorageStatusType type, final int storageNumber, final StorageMetric storageMetric) {
return type.getField() + storageNumber + storageMetric.getField();
}
private String getLabel(final StorageStatusType type, final CharSequence name, final StorageMetric storageMetric) {
return type.getLabel() + " (" + name + ") " + storageMetric.getLabel();
}
}

View File

@ -0,0 +1,57 @@
/*
* 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 java.util.Arrays;
import java.util.Optional;
enum StorageStatusType {
CONTENT((short)0, "contentStorage", "Content Repository"),
PROVENANCE((short)1, "provenanceStorage", "Provenance Repository");
private final short id;
private final String field;
private final String label;
StorageStatusType(final short id, final String field, final String label) {
this.id = id;
this.field = field;
this.label = label;
}
static StorageStatusType getById(final int id) {
final Optional<StorageStatusType> result = Arrays.stream(StorageStatusType.values()).filter(storageStatusType -> storageStatusType.getId() == id).findFirst();
if (result.isEmpty()) {
throw new IllegalArgumentException("Unknown storage type id " + id);
}
return result.get();
}
short getId() {
return id;
}
String getField() {
return field;
}
String getLabel() {
return label;
}
}

View File

@ -12,4 +12,4 @@
# 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.EmbeddedQuestDbStatusHistoryRepository
org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepository

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller.status.history;
import org.apache.nifi.controller.status.history.questdb.EmbeddedQuestDbStatusHistoryRepository;
import org.apache.nifi.util.NiFiProperties;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
@ -37,9 +38,9 @@ public abstract class AbstractEmbeddedQuestDbStatusHistoryRepositoryTest extends
protected static final int PREFERRED_DATA_POINTS = 1000;
protected static final int DAYS_TO_KEEP_DATA = 7;
protected static final long PERSIST_FREQUENCY = 50; //200 milliseconds
protected static final String PERSIST_FREQUENCY = "200 ms"; // 200 milliseconds
protected EmbeddedQuestDbStatusHistoryRepository repository;
protected StatusHistoryRepository repository;
@TempDir
private Path temporaryDirectory;
@ -63,7 +64,7 @@ public abstract class AbstractEmbeddedQuestDbStatusHistoryRepositoryTest extends
repository.shutdown();
}
private EmbeddedQuestDbStatusHistoryRepository startRepository() {
private StatusHistoryRepository startRepository() {
final NiFiProperties niFiProperties = Mockito.mock(NiFiProperties.class);
Mockito.when(niFiProperties.getIntegerProperty(
@ -76,9 +77,12 @@ public abstract class AbstractEmbeddedQuestDbStatusHistoryRepositoryTest extends
NiFiProperties.DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_COMPONENT_DAYS)
).thenReturn(DAYS_TO_KEEP_DATA);
Mockito.when(niFiProperties.getQuestDbStatusRepositoryPath()).thenReturn(temporaryDirectory);
Mockito.when(niFiProperties.getQuestDbStatusRepositoryPersistBatchSize()).thenReturn(Integer.parseInt(NiFiProperties.DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_BATCH_SIZE));
final EmbeddedQuestDbStatusHistoryRepository testSubject = new EmbeddedQuestDbStatusHistoryRepository(niFiProperties, PERSIST_FREQUENCY);
Mockito.when(niFiProperties.getQuestDbStatusRepositoryPath()).thenReturn(temporaryDirectory.toAbsolutePath().toString());
Mockito.when(niFiProperties.getQuestDbStatusRepositoryPersistFrequency()).thenReturn(PERSIST_FREQUENCY);
final StatusHistoryRepository testSubject = new EmbeddedQuestDbStatusHistoryRepository(niFiProperties);
testSubject.start();
return testSubject;
}

View File

@ -23,7 +23,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class EmbeddedQuestDbStatusHistoryRepositoryForNodeTest extends AbstractEmbeddedQuestDbStatusHistoryRepositoryTest {
private static final long ZERO_BYTES = 0L;
private static final int ZERO_COUNT = 0;
@Test

View File

@ -0,0 +1,62 @@
<?xml version="1.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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-questdb-bundle</artifactId>
<version>2.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-questdb</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>
<version>2.0.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>
<dependency>
<groupId>org.questdb</groupId>
<artifactId>questdb</artifactId>
<version>7.3.7</version>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-junit-jupiter</artifactId>
</dependency>
<dependency>
<groupId>org.springframework</groupId>
<artifactId>spring-core</artifactId>
</dependency>
<dependency>
<groupId>org.springframework.retry</groupId>
<artifactId>spring-retry</artifactId>
<version>2.0.5</version>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.questdb;
/**
* Wraps the necessary services of QuestDb.
*/
public interface Client {
/**
* Executes a query not caring about the possible return values. Primarily for DDL commands.
*
* @param query The QuestDB query to execute.
*/
void execute(String query) throws DatabaseException;
/**
* Inserts new rows into the database using a source object for specifying the inserted data.
*
* @param tableName The target table.
* @param rowSource Specifies the row data to be inserted.
*/
void insert(
String tableName,
InsertRowDataSource rowSource
) throws DatabaseException;
/**
*
* Queries information from the database which will be handled by a row processor. This row processor depending on the
* implementation might create an entry for every row in the query result ({@code QuestDbRowProcessor#forMapping} or
* potentially create an aggregate.
*
* @param query The query string.
* @param rowProcessor Maps the query result into a presentation format.
*
* @return The result of the query.
*
* @param <T> Result type.
*/
<T> T query(
String query,
QueryResultProcessor<T> rowProcessor
) throws DatabaseException;
/**
* Terminates the client. After {@code disconnect} is called, answer for other calls is not guaranteed.
*/
void disconnect() throws DatabaseException;
}

View File

@ -0,0 +1,32 @@
/*
* 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.questdb;
public class DatabaseException extends Exception {
public DatabaseException(final Throwable cause) {
super(cause);
}
public DatabaseException(final String message) {
super(message);
}
public DatabaseException(final String message, final Throwable cause) {
super(message, cause);
}
}

View File

@ -14,25 +14,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.history.storage;
package org.apache.nifi.questdb;
import org.apache.nifi.controller.status.NodeStatus;
import org.apache.nifi.controller.status.history.StatusHistory;
import java.time.Instant;
import java.io.Closeable;
/**
* Readable status storage for the node status entries.
* Provides access to database via distributing clients. Also responsible to ensure the health of the database connection
* and database if possible.
*/
public interface NodeStatusStorage extends StatusStorage<NodeStatus> {
public interface DatabaseManager extends Closeable {
/**
* @return A client to execute queries against the managed database instance.
*/
Client acquireClient();
/**
* 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.
* Starts maintenance of the database. Necessary initialization step for proper use.
*/
StatusHistory read(Instant start, Instant end);
void init();
/**
* Finishes maintenance of the database. After calling, manager does not guarantee any connection with the database.
*/
void close();
}

View File

@ -14,25 +14,21 @@
* 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;
package org.apache.nifi.questdb;
import java.time.Instant;
/**
* Readable status storage for garbage collection status entries.
* Used by {@code InsertRowDataSource} in order to populate the rows to insert. Calling {@code #initializeRow} before adding
* fields is mandatory. Every call initializes a new row thus it is not possible to effect the row data prior to the call.
*/
public interface GarbageCollectionStatusStorage extends StatusStorage<GarbageCollectionStatus> {
public interface InsertRowContext {
InsertRowContext initializeRow(Instant captured);
/**
* 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);
InsertRowContext addLong(int position, long value);
InsertRowContext addInt(int position, int value);
InsertRowContext addShort(int position, short value);
InsertRowContext addString(int position, String value);
InsertRowContext addInstant(int position, Instant value);
InsertRowContext addTimestamp(int position, long value);
}

View File

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

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.questdb;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
import java.util.HashMap;
import java.util.Map;
import java.util.function.BiFunction;
public interface QueryResultProcessor<R> {
Map<Class<?>, BiFunction<Integer, QueryRowContext, ?>> ENTRY_FILLERS = new HashMap<>() {{
put(Integer.class, (p, r) -> r.getInt(p));
put(Long.class, (p, r) -> r.getLong(p));
put(String.class, (p, r) -> r.getString(p));
put(Instant.class, (p, r) -> microsToInstant(r.getTimestamp(p)));
}};
private static Instant microsToInstant(final long micros) {
return Instant.EPOCH.plus(micros, ChronoUnit.MICROS);
}
void processRow(QueryRowContext context);
R getResult();
}

View File

@ -0,0 +1,25 @@
/*
* 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.questdb;
public interface QueryRowContext {
int getInt(int position);
long getLong(int position);
long getTimestamp(int position);
short getShort(int position);
String getString(int position);
}

View File

@ -0,0 +1,26 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.DatabaseException;
final class ClientDisconnectedException extends DatabaseException {
ClientDisconnectedException(final String message) {
super(message);
}
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
final class ConditionAwareClient implements Client {
public static final String CONDITION_ERROR = "Condition is not met, cannot call the enveloped client";
private final Client client;
private final Condition condition;
ConditionAwareClient(final Condition condition, final Client client) {
this.client = client;
this.condition = condition;
}
@Override
public void execute(final String query) throws DatabaseException {
if (condition.check()) {
client.execute(query);
} else {
throw new ConditionFailedException(CONDITION_ERROR);
}
}
@Override
public void insert(final String tableName, final InsertRowDataSource rowSource) throws DatabaseException {
if (condition.check()) {
client.insert(tableName, rowSource);
} else {
throw new ConditionFailedException(CONDITION_ERROR);
}
}
@Override
public <T> T query(final String query, final QueryResultProcessor<T> rowProcessor) throws DatabaseException {
if (condition.check()) {
return client.query(query, rowProcessor);
} else {
throw new ConditionFailedException(CONDITION_ERROR);
}
}
@Override
public void disconnect() throws DatabaseException {
client.disconnect();
}
@FunctionalInterface
interface Condition {
boolean check();
}
@Override
public String toString() {
return "ConditionAwareQuestDbClient{" +
"client=" + client +
", condition=" + condition +
'}';
}
}

View File

@ -0,0 +1,26 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.DatabaseException;
final class ConditionFailedException extends DatabaseException {
ConditionFailedException(final String message) {
super(message);
}
}

View File

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

View File

@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.questdb.embedded;
import io.questdb.cairo.sql.Record;
import io.questdb.cairo.sql.RecordCursor;
import org.apache.nifi.questdb.QueryRowContext;
final class CursorBasedQueryRowContext implements QueryRowContext {
private final RecordCursor cursor;
private Record actualRecord;
CursorBasedQueryRowContext(final RecordCursor cursor) {
this.cursor = cursor;
}
@Override
public int getInt(final int position) {
return actualRecord.getInt(position);
}
@Override
public long getLong(final int position) {
return actualRecord.getLong(position);
}
@Override
public long getTimestamp(final int position) {
return actualRecord.getTimestamp(position);
}
@Override
public short getShort(final int position) {
return actualRecord.getShort(position);
}
@Override
public String getString(final int position) {
return String.valueOf(actualRecord.getSym(position));
}
boolean hasNext() {
return cursor.hasNext();
}
void moveToNext() {
actualRecord = cursor.getRecord();
}
}

View File

@ -0,0 +1,151 @@
/*
* 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.questdb.embedded;
import io.questdb.cairo.CairoEngine;
import io.questdb.cairo.CairoError;
import io.questdb.cairo.TableToken;
import io.questdb.cairo.TableWriter;
import io.questdb.cairo.sql.RecordCursor;
import io.questdb.cairo.sql.RecordCursorFactory;
import io.questdb.griffin.CompiledQuery;
import io.questdb.griffin.SqlCompiler;
import io.questdb.griffin.SqlCompilerFactoryImpl;
import io.questdb.griffin.SqlException;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.mp.SCSequence;
import io.questdb.mp.TimeoutBlockingWaitStrategy;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Supplier;
final class EmbeddedClient implements Client {
private final static Logger LOGGER = LoggerFactory.getLogger(EmbeddedClient.class);
private final Supplier<CairoEngine> engine;
private final AtomicBoolean disconnected = new AtomicBoolean(false);
EmbeddedClient(final Supplier<CairoEngine> engine) {
this.engine = engine;
}
@Override
public void execute(final String query) throws DatabaseException {
checkConnectionState();
try (final SqlCompiler compiler = getCompiler()) {
final CompiledQuery compile = compiler.compile(query, getSqlExecutionContext());
compile.execute(new SCSequence(new TimeoutBlockingWaitStrategy(5, TimeUnit.SECONDS)));
} catch (final SqlException | CairoError e) {
throw new DatabaseException(e);
}
}
@Override
public void insert(
final String tableName,
final InsertRowDataSource rowDataSource
) throws DatabaseException {
checkConnectionState();
if (!rowDataSource.hasNextToInsert()) {
LOGGER.debug("No rows to insert into {}", tableName);
return;
}
final TableToken tableToken = engine.get().getTableTokenIfExists(tableName);
if (tableToken == null) {
throw new DatabaseException(String.format("Table Token for table [%s] not found", tableName));
}
try (
final TableWriter tableWriter = engine.get().getWriter(tableToken, "adding rows")
) {
final TableWriterBasedInsertRowContext context = new TableWriterBasedInsertRowContext(tableWriter);
while (rowDataSource.hasNextToInsert()) {
context.addRow(rowDataSource);
}
LOGGER.debug("Committing {} rows", tableWriter.getRowCount());
tableWriter.commit();
} catch (final Exception | CairoError e) {
// CairoError might be thrown in extreme cases, for example when no space left on the disk
throw new DatabaseException(e);
} finally {
engine.get().releaseInactive();
}
}
@Override
public <T> T query(final String query, final QueryResultProcessor<T> rowProcessor) throws DatabaseException {
checkConnectionState();
final CompiledQuery compiledQuery;
try (final SqlCompiler compiler = getCompiler()) {
compiledQuery = compiler.compile(query, getSqlExecutionContext());
} catch (final SqlException | CairoError e) {
throw new DatabaseException(e);
}
try (
final RecordCursorFactory factory = compiledQuery.getRecordCursorFactory();
final RecordCursor cursor = factory.getCursor(getSqlExecutionContext());
) {
final CursorBasedQueryRowContext rowContext = new CursorBasedQueryRowContext(cursor);
while ((rowContext.hasNext())) {
rowContext.moveToNext();
rowProcessor.processRow(rowContext);
}
return rowProcessor.getResult();
} catch (final Exception e) {
throw new DatabaseException(e);
}
}
@Override
public void disconnect() throws DatabaseException {
checkConnectionState();
disconnected.set(true);
LOGGER.info("Client disconnected");
}
private void checkConnectionState() throws DatabaseException {
if (disconnected.get()) {
throw new ClientDisconnectedException("The client is already disconnected");
}
}
private SqlCompiler getCompiler() {
return SqlCompilerFactoryImpl.INSTANCE.getInstance(engine.get());
}
private SqlExecutionContext getSqlExecutionContext() {
return SqlExecutionContextFactory.getInstance(engine.get());
}
}

View File

@ -0,0 +1,294 @@
/*
* 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.questdb.embedded;
import io.questdb.cairo.CairoConfiguration;
import io.questdb.cairo.CairoEngine;
import io.questdb.cairo.DefaultCairoConfiguration;
import io.questdb.cairo.TableToken;
import io.questdb.cairo.sql.TableRecordMetadata;
import org.apache.commons.lang3.concurrent.BasicThreadFactory;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.DatabaseManager;
import org.apache.nifi.util.file.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
final class EmbeddedDatabaseManager implements DatabaseManager {
private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedDatabaseManager.class);
private final String id = UUID.randomUUID().toString();
private final AtomicReference<EmbeddedDatabaseManagerStatus> state = new AtomicReference(EmbeddedDatabaseManagerStatus.UNINITIALIZED);
private final ReadWriteLock databaseStructureLock = new ReentrantReadWriteLock();
private final EmbeddedDatabaseManagerContext context;
private final AtomicReference<CairoEngine> engine = new AtomicReference<>();
private final List<ScheduledFuture<?>> scheduledFutures = new ArrayList<>();
private final ScheduledExecutorService scheduledExecutorService = Executors
.newScheduledThreadPool(2, new BasicThreadFactory.Builder().namingPattern("EmbeddedQuestDbManagerWorker-" + id + "-%d").build());
EmbeddedDatabaseManager(final EmbeddedDatabaseManagerContext context) {
this.context = context;
}
@Override
public void init() {
if (state.get() != EmbeddedDatabaseManagerStatus.UNINITIALIZED) {
throw new IllegalStateException("Manager is already initialized");
}
ensureDatabaseIsReady();
startRollover();
}
private void ensureDatabaseIsReady() {
boolean successful = false;
try {
databaseStructureLock.writeLock().lock();
state.set(EmbeddedDatabaseManagerStatus.REPAIRING);
try {
ensurePersistLocationIsAccessible();
ensureConnectionEstablished();
ensureTablesAreInPlaceAndHealthy();
successful = true;
} catch (final CorruptedDatabaseException e) {
boolean couldMoveOldToBackup = false;
try {
LOGGER.error("Database is corrupted. Recreation is triggered. Manager tries to move corrupted database files to the backup location: {}", context.getBackupLocation(), e);
final File backupFolder = new File(context.getBackupLocationAsPath().toFile(), "backup_" + System.currentTimeMillis());
FileUtils.ensureDirectoryExistAndCanAccess(context.getBackupLocationAsPath().toFile());
Files.move(context.getPersistLocationAsPath(), backupFolder.toPath());
couldMoveOldToBackup = true;
} catch (IOException ex) {
LOGGER.error("Could not create backup", ex);
}
if (!couldMoveOldToBackup) {
try {
FileUtils.deleteFile(context.getPersistLocationAsPath().toFile(), true);
couldMoveOldToBackup = true;
} catch (IOException ex) {
LOGGER.error("Could not clean up corrupted database", ex);
}
}
if (couldMoveOldToBackup) {
try {
ensurePersistLocationIsAccessible();
ensureConnectionEstablished();
ensureTablesAreInPlaceAndHealthy();
successful = true;
} catch (CorruptedDatabaseException ex) {
LOGGER.error("Could not create backup", ex);
}
}
}
} finally {
state.set(successful? EmbeddedDatabaseManagerStatus.HEALTHY : EmbeddedDatabaseManagerStatus.CORRUPTED);
if (!successful) {
engine.set(null);
}
databaseStructureLock.writeLock().unlock();
}
}
private void ensurePersistLocationIsAccessible() throws CorruptedDatabaseException {
try {
FileUtils.ensureDirectoryExistAndCanAccess(context.getPersistLocationAsPath().toFile());
} catch (final Exception e) {
throw new CorruptedDatabaseException(String.format("Database directory creation failed [%s]", context.getPersistLocationAsPath()), e);
}
}
private void ensureConnectionEstablished() throws CorruptedDatabaseException {
if (engine.get() != null) {
engine.getAndSet(null).close();
}
final String absolutePath = context.getPersistLocationAsPath().toFile().getAbsolutePath();
final CairoConfiguration configuration = new DefaultCairoConfiguration(absolutePath);
try {
final CairoEngine engine = new CairoEngine(configuration);
LOGGER.info("Database connection successful [{}]", absolutePath);
this.engine.set(engine);
} catch (final Exception e) {
throw new CorruptedDatabaseException(String.format("Database connection failed [%s]", absolutePath), e);
}
}
private void ensureTablesAreInPlaceAndHealthy() throws CorruptedDatabaseException {
final Map<String, File> databaseFiles = Arrays.stream(context.getPersistLocationAsPath().toFile().listFiles())
.collect(Collectors.toMap(f -> f.getAbsolutePath().substring(context.getPersistLocationAsPath().toFile().getAbsolutePath().length() + 1), f -> f));
final Client client = getUnmanagedClient();
try {
for (final ManagedTableDefinition tableDefinition : context.getTableDefinitions()) {
if (!databaseFiles.containsKey(tableDefinition.getName())) {
try {
LOGGER.debug("Creating table {}", tableDefinition.getName());
client.execute(tableDefinition.getDefinition());
LOGGER.debug("Table {} is created", tableDefinition.getName());
} catch (DatabaseException e) {
throw new CorruptedDatabaseException(String.format("Creating table [%s] has failed", tableDefinition.getName()), e);
}
} else if (!databaseFiles.get(tableDefinition.getName()).isDirectory()) {
throw new CorruptedDatabaseException(String.format("Table %s cannot be created because there is already a file exists with the given name", tableDefinition.getName()));
}
}
// Checking if tables are healthy.
for (final ManagedTableDefinition tableDefinition : context.getTableDefinitions()) {
try {
final TableToken tableToken = this.engine.get().getTableTokenIfExists(tableDefinition.getName());
final TableRecordMetadata metadata = this.engine.get().getSequencerMetadata(tableToken);
metadata.close();
client.execute(String.format("SELECT * FROM %S LIMIT 1", tableDefinition.getName()));
} catch (final Exception e) {
throw new CorruptedDatabaseException(e);
}
}
} finally {
try {
client.disconnect();
} catch (DatabaseException e) {
throw new CorruptedDatabaseException(e);
}
}
}
private void startRollover() {
final RolloverWorker rolloverWorker = new RolloverWorker(acquireClient(), context.getTableDefinitions());
final ScheduledFuture<?> rolloverFuture = scheduledExecutorService.scheduleWithFixedDelay(
rolloverWorker, context.getRolloverFrequency().toMillis(), context.getRolloverFrequency().toMillis(), TimeUnit.MILLISECONDS);
scheduledFutures.add(rolloverFuture);
LOGGER.debug("Rollover started");
}
private void stopRollover() {
LOGGER.debug("Rollover shutdown started");
int cancelCompleted = 0;
int cancelFailed = 0;
for (final ScheduledFuture<?> scheduledFuture : scheduledFutures) {
final boolean cancelled = scheduledFuture.cancel(true);
if (cancelled) {
cancelCompleted++;
} else {
cancelFailed++;
}
}
LOGGER.debug("Rollover shutdown task cancellation status: completed [{}] failed [{}]", cancelCompleted, cancelFailed);
final List<Runnable> tasks = scheduledExecutorService.shutdownNow();
LOGGER.debug("Rollover Scheduled Task Service shutdown remaining tasks [{}]", tasks.size());
}
private Client getUnmanagedClient() {
return new EmbeddedClient(() -> engine.get());
}
public Client acquireClient() {
checkIfManagerIsInitialised();
final Client fallback = new NoOpClient();
if (state.get() == EmbeddedDatabaseManagerStatus.CORRUPTED) {
LOGGER.error("The database is corrupted: Status History will not be stored");
return fallback;
}
final LockedClient lockedClient = new LockedClient(
databaseStructureLock.readLock(),
context.getLockAttemptTime(),
new ConditionAwareClient(() -> state.get() == EmbeddedDatabaseManagerStatus.HEALTHY, getUnmanagedClient())
);
return RetryingClient.getInstance(context.getNumberOfAttemptedRetries(), this::errorAction, lockedClient, fallback);
}
private void checkIfManagerIsInitialised() {
if (state.get() == EmbeddedDatabaseManagerStatus.UNINITIALIZED) {
throw new IllegalStateException("The state of the database manager is not initialized");
}
}
private void errorAction(final int attemptNumber, final Throwable throwable) {
if (shouldRestoreDatabase(attemptNumber, throwable)) {
LOGGER.error("Database manager tries to restore database after the first failed attempt if necessary");
ensureDatabaseIsReady();
} else {
LOGGER.warn("Error happened at attempt: {}", attemptNumber, throwable);
}
}
private boolean shouldRestoreDatabase(final int attemptNumber, final Throwable throwable) {
if (state.get() == EmbeddedDatabaseManagerStatus.CORRUPTED
|| state.get() == EmbeddedDatabaseManagerStatus.CLOSED
) {
return false;
}
if (throwable instanceof ConditionFailedException
|| throwable instanceof LockUnsuccessfulException
|| throwable instanceof ClientDisconnectedException
) {
return false;
}
return attemptNumber == 1;
}
@Override
public void close() {
databaseStructureLock.writeLock().lock();
checkIfManagerIsInitialised();
stopRollover();
state.set(EmbeddedDatabaseManagerStatus.CLOSED);
final CairoEngine engine = this.engine.get();
if (engine != null) {
engine.close();
}
databaseStructureLock.writeLock().unlock();
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.DatabaseManager;
import org.apache.nifi.questdb.rollover.RolloverStrategy;
import java.io.File;
import java.time.Duration;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
public final class EmbeddedDatabaseManagerBuilder {
private SimpleEmbeddedDatabaseManagerContext context;
private EmbeddedDatabaseManagerBuilder(final String persistPath) {
this.context = new SimpleEmbeddedDatabaseManagerContext();
context.setPersistLocation(persistPath);
}
public EmbeddedDatabaseManagerBuilder lockAttemptTime(final int lockAttemptTime, final TimeUnit lockAttemptTimeUnit) {
context.setLockAttemptDuration(Duration.of(lockAttemptTime, lockAttemptTimeUnit.toChronoUnit()));
return this;
}
public EmbeddedDatabaseManagerBuilder rolloverFrequency(final int rolloverFrequency, final TimeUnit rolloverFrequencyTimeUnit) {
context.setRolloverFrequencyDuration(Duration.of(rolloverFrequency, rolloverFrequencyTimeUnit.toChronoUnit()));
return this;
}
public EmbeddedDatabaseManagerBuilder numberOfAttemptedRetries(final int numberOfAttemptedRetries) {
context.setNumberOfAttemptedRetries(numberOfAttemptedRetries);
return this;
}
public EmbeddedDatabaseManagerBuilder backupLocation(final String backupLocation) {
context.setBackupLocation(backupLocation);
return this;
}
public EmbeddedDatabaseManagerBuilder addTable(final String name, final String definition) {
return addTable(name, definition, RolloverStrategy.keep());
}
public EmbeddedDatabaseManagerBuilder addTable(final String name, final String definition, final RolloverStrategy rolloverStrategy) {
context.addTableDefinition(new ManagedTableDefinition(name, definition, rolloverStrategy));
return this;
}
public DatabaseManager build() {
Objects.requireNonNull(context.getLockAttemptTime(), "Lock attempt must be specified");
if (context.getLockAttemptTime().toMillis() <= 0) {
throw new IllegalArgumentException("Lock attempt time must be bigger than 0");
}
Objects.requireNonNull(context.getRolloverFrequency(), "Rollover frequency must be specified");
if (context.getRolloverFrequency().toMillis() <= 0) {
throw new IllegalArgumentException("Rollover frequency must be bigger than 0");
}
if (context.getNumberOfAttemptedRetries() < 1) {
throw new IllegalArgumentException("Number of attempted retries must be at least 1");
}
if (context.getTableDefinitions().isEmpty()) {
throw new IllegalArgumentException("There must be at least on table specified");
}
if (context.getBackupLocation() == null) {
context.setBackupLocation(new File(context.getPersistLocationAsPath().toFile().getParentFile(), "questDbBackup").getAbsolutePath());
}
final DatabaseManager result = new EmbeddedDatabaseManager(context);
result.init();
return result;
}
public static EmbeddedDatabaseManagerBuilder builder(final String persistPath) {
return new EmbeddedDatabaseManagerBuilder(persistPath);
}
}

View File

@ -0,0 +1,36 @@
/*
* 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.questdb.embedded;
import java.nio.file.Path;
import java.time.Duration;
import java.util.Set;
interface EmbeddedDatabaseManagerContext {
String getPersistLocation();
Path getPersistLocationAsPath();
String getBackupLocation();
Path getBackupLocationAsPath();
int getNumberOfAttemptedRetries();
Duration getLockAttemptTime();
Duration getRolloverFrequency();
Set<ManagedTableDefinition> getTableDefinitions();
}

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.questdb.embedded;
enum EmbeddedDatabaseManagerStatus {
/**
* Starting status. In this state the manager is not ready to be used. Method {@code QuestDbManager#init} should lead the manager out from this state.
*/
UNINITIALIZED,
/**
* The manager (and the enveloped database) is considered healthy and ready to be used.
*/
HEALTHY,
/**
* The database is in an unexpected state but the manager tries to resolve it. Might end up as {@code EmbeddedQuestDbManagerStatus.HEALTHY}.
*/
REPAIRING,
/**
* The database is in an unexpected state and the manager is not capable to resolve it. This is considered as a "final state".
*/
CORRUPTED,
/**
* The manager is considered shut down and the database is not eligible to work with. This is considered as a "final state"
*/
CLOSED
}

View File

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

View File

@ -0,0 +1,105 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Duration;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
final class LockedClient implements Client {
private static final Logger LOGGER = LoggerFactory.getLogger(LockedClient.class);
private final Lock lock;
private final Duration lockAttemptDuration;
private final Client client;
LockedClient(final Lock lock, final Duration lockAttemptDuration, final Client client) {
this.lock = lock;
this.lockAttemptDuration = lockAttemptDuration;
this.client = client;
}
@Override
public void execute(final String query) throws DatabaseException {
lockedOperation(() -> {
client.execute(query);
return null;
});
}
@Override
public void insert(final String tableName, final InsertRowDataSource rowSource) throws DatabaseException {
lockedOperation(() -> {
client.insert(tableName, rowSource);
return null;
});
}
@Override
public <T> T query(final String query, final QueryResultProcessor<T> rowProcessor) throws DatabaseException {
return lockedOperation(() -> client.query(query, rowProcessor));
}
@Override
public void disconnect() throws DatabaseException {
client.disconnect();
}
private <R> R lockedOperation(final Callable<R> operation) throws DatabaseException {
LOGGER.debug("Start locking client {}", client.toString());
try {
if (!lock.tryLock(lockAttemptDuration.toMillis(), TimeUnit.MILLISECONDS)) {
throw new LockUnsuccessfulException("Could not lock read lock on the database");
}
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
throw new LockUnsuccessfulException(e);
}
try {
LOGGER.debug("Successfully locked client {}", client);
return operation.call();
} catch (final DatabaseException e) {
LOGGER.error("Locked operation was unsuccessful", e);
throw e;
} catch (final Exception e) {
LOGGER.error("Locked operation was unsuccessful", e);
throw new DatabaseException(e);
} finally {
lock.unlock();
LOGGER.debug("Unlocked client {}", client);
}
}
@Override
public String toString() {
return "LockedQuestDbClient{" +
"lock=" + lock +
", lockAttemptTime=" + lockAttemptDuration +
", client=" + client +
'}';
}
}

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.questdb.embedded;
import org.apache.nifi.questdb.rollover.RolloverStrategy;
import java.util.Objects;
public final class ManagedTableDefinition {
private final String name;
private final String definition;
private final RolloverStrategy rolloverStrategy;
public ManagedTableDefinition(final String name, final String definition, final RolloverStrategy rolloverStrategy) {
this.name = name;
this.definition = definition;
this.rolloverStrategy = rolloverStrategy;
}
public String getName() {
return name;
}
public String getDefinition() {
return definition;
}
public RolloverStrategy getRolloverStrategy() {
return rolloverStrategy;
}
@Override
public boolean equals(final Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
final ManagedTableDefinition that = (ManagedTableDefinition) o;
return Objects.equals(name, that.name) && Objects.equals(definition, that.definition) && Objects.equals(rolloverStrategy, that.rolloverStrategy);
}
@Override
public int hashCode() {
return Objects.hash(name, definition, rolloverStrategy);
}
}

View File

@ -14,29 +14,35 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.history.storage;
package org.apache.nifi.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List;
final class NoOpClient implements Client {
private static final Logger LOGGER = LoggerFactory.getLogger(NoOpClient.class);
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 execute(final String query) {
LOGGER.debug("Execute: {}", query);
}
@Override
public void run() {
try {
bufferedWriterList.forEach(BufferedEntryWriter::flush);
} catch (final Exception e) {
LOGGER.error("Flush Buffered Writer failed", e);
}
public void insert(final String tableName, final InsertRowDataSource rowSource) {
LOGGER.debug("Inserting rows to Table {}", tableName);
}
@Override
public <T> T query(final String query, final QueryResultProcessor<T> rowProcessor) {
LOGGER.debug("Querying: {}", query);
return rowProcessor.getResult();
}
@Override
public void disconnect() {
LOGGER.debug("Disconnecting");
}
}

View File

@ -0,0 +1,124 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.springframework.retry.RecoveryCallback;
import org.springframework.retry.RetryCallback;
import org.springframework.retry.RetryContext;
import org.springframework.retry.RetryListener;
import org.springframework.retry.support.RetryTemplate;
import java.util.function.BiConsumer;
final class RetryingClient implements Client {
private final RetryTemplate retryTemplate;
private final Client client;
private final Client fallbackClient;
private RetryingClient(final RetryTemplate retryTemplate, final Client client, final Client fallbackClient) {
this.retryTemplate = retryTemplate;
this.client = client;
this.fallbackClient = fallbackClient;
}
@Override
public void execute(final String query) throws DatabaseException {
retryTemplate.execute(
new RetryWhenConnected<>() {
@Override
public Void executeWithRetry(final RetryContext context) throws DatabaseException {
client.execute(query);
return null;
}
},
(RecoveryCallback<Void>) context -> {
fallbackClient.execute(query);
return null;
}
);
}
@Override
public void insert(final String tableName, final InsertRowDataSource rowSource) throws DatabaseException {
retryTemplate.execute(
new RetryWhenConnected<>() {
@Override
public Void executeWithRetry(final RetryContext context) throws DatabaseException {
client.insert(tableName, rowSource);
return null;
}
},
(RecoveryCallback<Void>) context -> {
fallbackClient.insert(tableName, rowSource);
return null;
}
);
}
@Override
public <T> T query(final String query, final QueryResultProcessor<T> rowProcessor) throws DatabaseException {
return retryTemplate.execute(
new RetryWhenConnected<>() {
@Override
public T executeWithRetry(final RetryContext context) throws DatabaseException {
return client.query(query, rowProcessor);
}
},
context -> fallbackClient.query(query, rowProcessor)
);
}
@Override
public void disconnect() throws DatabaseException {
client.disconnect();
}
private static abstract class RetryWhenConnected<R> implements RetryCallback<R, DatabaseException> {
@Override
public R doWithRetry(final RetryContext context) throws DatabaseException {
try {
return executeWithRetry(context);
} catch (final ClientDisconnectedException e) {
context.setExhaustedOnly();
throw e;
}
}
public abstract R executeWithRetry(final RetryContext context) throws DatabaseException;
}
static RetryingClient getInstance(final int numberOfRetries, final BiConsumer<Integer, Throwable> errorAction, final Client client, final Client fallbackClient) {
final RetryListener listener = new RetryListener() {
@Override
public <T, E extends Throwable> void onError(final RetryContext context, final RetryCallback<T, E> callback, final Throwable throwable) {
errorAction.accept(context.getRetryCount(), throwable);
}
};
final RetryTemplate retryTemplate = RetryTemplate.builder()
.maxAttempts(numberOfRetries + 1)
.fixedBackoff(50)
.withListener(listener)
.build();
return new RetryingClient(retryTemplate, client, fallbackClient);
}
}

View File

@ -0,0 +1,49 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashSet;
import java.util.Set;
final class RolloverWorker implements Runnable {
private static final Logger LOGGER = LoggerFactory.getLogger(RolloverWorker.class);
private final Client client;
private final Set<ManagedTableDefinition> tableDefinitions = new HashSet<>();
RolloverWorker(final Client client, final Set<ManagedTableDefinition> tableDefinitions) {
this.client = client;
this.tableDefinitions.addAll(tableDefinitions);
}
@Override
public void run() {
LOGGER.debug("Rollover started");
for (final ManagedTableDefinition tableDefinition : tableDefinitions) {
LOGGER.debug("Rollover started for Table {}", tableDefinition.getName());
tableDefinition.getRolloverStrategy().rollOver(client, tableDefinition.getName());
LOGGER.debug("Rollover completed for Table {}", tableDefinition.getName());
}
LOGGER.debug("Rollover completed");
}
}

View File

@ -0,0 +1,109 @@
/*
* 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.questdb.embedded;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.time.Duration;
import java.util.HashSet;
import java.util.Set;
final class SimpleEmbeddedDatabaseManagerContext implements EmbeddedDatabaseManagerContext {
final private Set<ManagedTableDefinition> tableDefinitions = new HashSet<>();
private String persistLocation;
private String backupLocation;
private int numberOfAttemptedRetries;
private Duration lockAttemptDuration;
private Duration rolloverFrequnecyDuration;
@Override
public String getPersistLocation() {
return persistLocation;
}
@Override
public Path getPersistLocationAsPath() {
return Paths.get(getPersistLocation());
}
@Override
public String getBackupLocation() {
return backupLocation;
}
@Override
public Path getBackupLocationAsPath() {
return Paths.get(getBackupLocation());
}
@Override
public int getNumberOfAttemptedRetries() {
return numberOfAttemptedRetries;
}
@Override
public Duration getLockAttemptTime() {
return lockAttemptDuration;
}
@Override
public Duration getRolloverFrequency() {
return rolloverFrequnecyDuration;
}
@Override
public Set<ManagedTableDefinition> getTableDefinitions() {
return tableDefinitions;
}
void setPersistLocation(final String persistLocation) {
this.persistLocation = persistLocation;
}
public void setBackupLocation(final String backupLocation) {
this.backupLocation = backupLocation;
}
void setNumberOfAttemptedRetries(final int numberOfAttemptedRetries) {
this.numberOfAttemptedRetries = numberOfAttemptedRetries;
}
void setLockAttemptDuration(final Duration lockAttemptDuration) {
this.lockAttemptDuration = lockAttemptDuration;
}
void setRolloverFrequencyDuration(final Duration rolloverFreqencyDuration) {
this.rolloverFrequnecyDuration = rolloverFreqencyDuration;
}
void addTableDefinition(final ManagedTableDefinition tableDefinition) {
tableDefinitions.add(tableDefinition);
}
@Override
public String toString() {
return "SimpleEmbeddedDatabaseManagerContext{" +
"tableDefinitions=" + tableDefinitions +
", persistLocation='" + persistLocation + '\'' +
", backupLocation='" + backupLocation + '\'' +
", numberOfAttemptedRetries=" + numberOfAttemptedRetries +
", lockAttemptDuration=" + lockAttemptDuration +
", rolloverFrequnecyDuration=" + rolloverFrequnecyDuration +
'}';
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.status.history.questdb;
package org.apache.nifi.questdb.embedded;
import io.questdb.cairo.CairoEngine;
import io.questdb.cairo.security.AllowAllSecurityContext;
@ -26,7 +26,7 @@ final class SqlExecutionContextFactory {
// Not to be instantiated
}
public static SqlExecutionContext getInstance(final CairoEngine engine) {
static SqlExecutionContext getInstance(final CairoEngine engine) {
return new SqlExecutionContextImpl(engine, 1).with(AllowAllSecurityContext.INSTANCE, null);
}
}

View File

@ -0,0 +1,94 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.questdb.embedded;
import io.questdb.cairo.TableWriter;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowContext;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
final class TableWriterBasedInsertRowContext implements InsertRowContext {
public static final Logger LOGGER = LoggerFactory.getLogger(TableWriterBasedInsertRowContext.class);
private final TableWriter tableWriter;
private TableWriter.Row actualRow;
TableWriterBasedInsertRowContext(final TableWriter tableWriter) {
this.tableWriter = tableWriter;
}
void addRow(final InsertRowDataSource rowDataSource) throws DatabaseException {
rowDataSource.fillRowData(this);
actualRow.append();
actualRow = null;
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Appending new row to the table writer: {}", actualRow.toString());
}
}
@Override
public InsertRowContext initializeRow(final Instant captured) {
actualRow = tableWriter.newRow(instantToMicros(captured));
return this;
}
@Override
public InsertRowContext addLong(final int position, final long value) {
actualRow.putLong(position, value);
return this;
}
@Override
public InsertRowContext addInt(final int position, final int value) {
actualRow.putInt(position, value);
return this;
}
@Override
public InsertRowContext addShort(final int position, final short value) {
actualRow.putShort(position, value);
return this;
}
@Override
public InsertRowContext addString(final int position, final String value) {
actualRow.putSym(position, value);
return this;
}
@Override
public InsertRowContext addInstant(final int position, final Instant value) {
actualRow.putTimestamp(position, instantToMicros(value));
return this;
}
@Override
public InsertRowContext addTimestamp(final int position, final long value) {
actualRow.putTimestamp(position, value);
return this;
}
private static long instantToMicros(final Instant instant) {
return ChronoUnit.MICROS.between(Instant.EPOCH, instant);
}
}

View File

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.questdb.mapping;
import org.apache.nifi.questdb.QueryResultProcessor;
import java.util.List;
import java.util.function.BiConsumer;
public interface RequestMapping<T> {
T getNewInstance();
int getNumberOfFields();
Class<?> getFieldType(int position);
BiConsumer<T, Object> getMapping(int position);
static <T> QueryResultProcessor<List<T>> getResultProcessor(final RequestMapping<T> mapping) {
return new RequestMappingBasedQueryResultProcessor<>(mapping);
}
}

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.questdb.mapping;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.QueryRowContext;
import java.util.ArrayList;
import java.util.List;
import java.util.function.BiFunction;
final class RequestMappingBasedQueryResultProcessor<T> implements QueryResultProcessor<List<T>> {
private final RequestMapping<T> mapping;
private final List<T> result = new ArrayList<>();
RequestMappingBasedQueryResultProcessor(final RequestMapping<T> mapping) {
this.mapping = mapping;
}
@Override
public void processRow(final QueryRowContext context) {
final T entry = mapping.getNewInstance();
for (int position = 0; position < mapping.getNumberOfFields(); position++) {
if (!ENTRY_FILLERS.containsKey(mapping.getFieldType(position))) {
throw new IllegalArgumentException(String.format("Unknown field type \"%s\"", mapping.getFieldType(position)));
}
final BiFunction<Integer, QueryRowContext, ?> integerRecordBiFunction = ENTRY_FILLERS.get(mapping.getFieldType(position));
mapping.getMapping(position).accept(entry, integerRecordBiFunction.apply(position, context));
}
result.add(entry);
}
@Override
public List<T> getResult() {
return result;
}
}

View File

@ -0,0 +1,67 @@
/*
* 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.questdb.mapping;
import org.apache.commons.lang3.tuple.Pair;
import java.time.Instant;
import java.util.LinkedList;
import java.util.List;
import java.util.function.BiConsumer;
import java.util.function.Supplier;
public final class RequestMappingBuilder<T> {
private final Supplier<T> factory;
private final List<Pair<Class<?>, BiConsumer<T, Object>>> fieldMappings = new LinkedList<>();
private RequestMappingBuilder(final Supplier<T> factory) {
this.factory = factory;
}
public <A> RequestMappingBuilder<T> addField(final Class<A> type, final BiConsumer<T, A> mapping) {
fieldMappings.add(Pair.of(type, (BiConsumer<T, Object>) mapping));
return this;
}
public RequestMappingBuilder<T> addStringField(final BiConsumer<T, String> mapping) {
return addField(String.class, mapping);
}
public RequestMappingBuilder<T> addInstantField(final BiConsumer<T, Instant> mapping) {
return addField(Instant.class, mapping);
}
public RequestMappingBuilder<T> addLongField(final BiConsumer<T, Long> mapping) {
return addField(Long.class, mapping);
}
public RequestMappingBuilder<T> addIntegerField(final BiConsumer<T, Integer> mapping) {
return addField(Integer.class, mapping);
}
public static <E> RequestMappingBuilder<E> of(final Supplier<E> factory) {
return new RequestMappingBuilder<>(factory);
}
public RequestMapping<T> build() {
if (fieldMappings.isEmpty()) {
throw new IllegalArgumentException("There must be at least one declared field");
}
return new SimpleRequestMapping<T>(factory, fieldMappings);
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.questdb.mapping;
import org.apache.commons.lang3.tuple.Pair;
import java.util.ArrayList;
import java.util.List;
import java.util.function.BiConsumer;
import java.util.function.Supplier;
final class SimpleRequestMapping<T> implements RequestMapping<T> {
private final Supplier<T> factory;
private final List<Pair<Class<?>, BiConsumer<T, Object>>> fieldMappings;
SimpleRequestMapping(
final Supplier<T> factory,
final List<Pair<Class<?>, BiConsumer<T, Object>>> fieldMappings
) {
this.factory = factory;
this.fieldMappings = new ArrayList<>(fieldMappings);
}
@Override
public T getNewInstance() {
return factory.get();
}
@Override
public int getNumberOfFields() {
return fieldMappings.size();
}
@Override
public Class<?> getFieldType(final int position) {
return fieldMappings.get(position).getKey();
}
@Override
public BiConsumer<T, Object> getMapping(final int position) {
return fieldMappings.get(position).getRight();
}
}

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.
*/
/**
* <p>
* This package wraps all QuestDB features utilized by NiFi. The content of the package serves as an adaptor and a facade
* for QuestDB API, encapsulating all the direct dependencies for QuestDB.
* </p>
*
* <p>
* This has multiple purposes such as:
* </p>
*
* <ul>
* <li>Making version changes of the underlying library easier.</li>
* <li>Preventing dependency sprawl.</li>
* <li>Hiding usage details.</li>
* </ul>
*
* <p>
* The main entry point for the <code>DatabaseManager</code> which is responsible for returning a <code>Client</code> which
* might be used for the NiFi code to interact with the QuestDB instance.
* </p>
*
* <h3>Embedded implementation</h3>
*
* <p>
* Currently the bundle supports connecting to an embedded QuestDB instance which is managed by the <code>EmbeddedDatabaseManager</code>.
* This includes creating the database and necessary tables and also rolling over old data. In order to up an embedded database a
* properly parametrized <code>EmbeddedDatabaseManagerContext</code> is necessary with the <code>ManagedTableDefinition</code>
* instances in place.
* </p>
*
* <p>
* Users can specify table definition and roll over strategy via <code>ManagedTableDefinition</code> instances. The manager then
* will ensure that the tables are created and not altered. It is not recommended to execute alterations on the tables from outside
* sources. Version management for table schemes is not currently supported.
* </p>
*
*
* <h3>Mapping</h3>
*
* <p>
* For ease of use, the bundle supports "mapping" between "rows" (representation of data within QuestDB) and "entries" (representation
* of data within NiFi) when executing queries. In order to utilize this capability a declerative description of the mapping is needed
* using the <code>RequestMappingBuilder</code>. It can be wrapped by <code>RequestMapping#getResultProcessor</code> and used as any
* <code>QueryResultProcessor</code>.
* </p>
*
* <h3>Error handling</h3>
*
* <p>
* The bundle provides error handling using various exceptions all derived from <code>DatabaseException</code>. The embedded implementation
* also provides a level of "restoration" capabilities in case the situation allows it. This can lead to situations where the existing database
* is moved out from it's place and a new instance is created! Also: in extreme cases (especially with storage related issues) the corrupted
* database instance might be deleted and/or the service falls back to a dummy mode, similar to a circuit breaker. Resolving these situations
* might need human intervention.
* </p>
*/
package org.apache.nifi.questdb;

View File

@ -0,0 +1,95 @@
/*
* 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.questdb.rollover;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.QueryRowContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.ZoneOffset;
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.function.Supplier;
final class DeleteOldRolloverStrategy implements RolloverStrategy {
private static final Logger LOGGER = LoggerFactory.getLogger(DeleteOldRolloverStrategy.class);
private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd").withZone(ZoneOffset.UTC);
private static final String DELETION_QUERY = "ALTER TABLE %s DROP PARTITION LIST '%s'";
// Distinct keyword is not recognized if the date mapping is not within an inner query
private static final String PARTITION_SELECTION_QUERY = "SELECT DISTINCT * FROM (SELECT (to_str(captured, 'yyyy-MM-dd')) AS partitionName FROM %s)";
private final Supplier<ZonedDateTime> timeSource;
private final int daysToKeep;
DeleteOldRolloverStrategy(final Supplier<ZonedDateTime> timeSource, final int daysToKeep) {
this.timeSource = timeSource;
this.daysToKeep = daysToKeep;
}
@Override
public void rollOver(final Client client, final String tableName) {
try {
final List<String> partitions = getPartitions(client, tableName);
final String oldestPartitionToKeep = getOldestPartitionToKeep();
// The last partition if exists, it is considered as "active partition" and cannot be deleted.
for (int i = 0; i < partitions.size() - 1; i++) {
final String partition = partitions.get(i);
if (oldestPartitionToKeep.compareTo(partition) > 0) {
try {
client.execute(String.format(DELETION_QUERY, tableName, partition));
LOGGER.debug("Dropping partition [{}] of table [{}] was successful", partition, tableName);
} catch (final Exception e) {
LOGGER.error("Dropping partition [{}] of table [{}] failed", partition, tableName, e);
}
}
}
} catch (final Exception e2) {
LOGGER.error("Rollover failed for table [{}]", tableName, e2);
}
}
private List<String> getPartitions(final Client client, final CharSequence tableName) throws Exception {
return client.query(String.format(PARTITION_SELECTION_QUERY, tableName), new PartitionQueryResultProcessor());
}
private String getOldestPartitionToKeep() {
final ZonedDateTime now = timeSource.get();
final ZonedDateTime utc = now.minusDays(daysToKeep).withZoneSameInstant(ZoneOffset.UTC);
return utc.format(DATE_FORMATTER);
}
private static class PartitionQueryResultProcessor implements QueryResultProcessor<List<String>> {
List<String> result = new LinkedList<>();
@Override
public void processRow(final QueryRowContext context) {
result.add(context.getString(0));
}
@Override
public List<String> getResult() {
Collections.sort(result);
return result;
}
}
}

View File

@ -0,0 +1,30 @@
/*
* 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.questdb.rollover;
import org.apache.nifi.questdb.Client;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
final class KeepAllRolloverStrategy implements RolloverStrategy {
Logger LOGGER = LoggerFactory.getLogger(KeepAllRolloverStrategy.class);
@Override
public void rollOver(final Client client, final String tableName) {
LOGGER.debug("Rollover executed for {}", tableName);
}
}

View File

@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.questdb.rollover;
import org.apache.nifi.questdb.Client;
import java.time.ZonedDateTime;
/**
* Defines a rollover strategy. The rollover strategy describes how the dataset withing a table should be handled in order
* to avoid constant growing without limitation.
*
* The instance is not unique to a given table but unique to a given behaviour. Might be used with multiple tables.
*/
public interface RolloverStrategy {
/**
* Executes the rollover strategy on a given table.
*
* @param client The client to connect the database.
* @param tableName The subject table.
*/
void rollOver(Client client, String tableName);
static RolloverStrategy keep() {
return new KeepAllRolloverStrategy();
}
static RolloverStrategy deleteOld(final int daysToKeep) {
return new DeleteOldRolloverStrategy(() -> ZonedDateTime.now(), daysToKeep);
}
}

View File

@ -0,0 +1,226 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.time.Duration;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.BiConsumer;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.SELECT_QUERY;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
// This test uses the {@code compile} method to cover different scenarios. It is expected that other methods behave the same
// but the coverage of the individual behaviour of the individual methods is provided by different tests.
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.LENIENT)
public class CompositeClientTest {
private static final int NUMBER_OF_RETRIES = 3;
private static final int NUMBER_OF_ATTEMPTS = NUMBER_OF_RETRIES + 1;
private static final Duration LOCK_ATTEMPT_DURATION = Duration.of(20, TimeUnit.MILLISECONDS.toChronoUnit());
@Mock
Client client;
@Mock
Client fallback;
@Mock
BiConsumer<Integer, Throwable> errorAction;
@Mock
ConditionAwareClient.Condition condition;
private ReadWriteLock databaseStructureLock;
private LockedClient lockedClientSpy;
private Client testSubject;
@BeforeEach
public void setUp() {
databaseStructureLock = new ReentrantReadWriteLock();
testSubject = getTestSubject();
when(condition.check()).thenReturn(true);
}
@Test
public void testHappyPath() throws DatabaseException {
testSubject.execute(SELECT_QUERY);
assertWrappedClientIsCalled(1);
assertFallbackIsNotCalled();
assertErrorActionIsNotCalled();
}
@Test
public void testSingleErrorDuringExecution() throws DatabaseException {
doThrow(new DatabaseException("Test")).doNothing().when(client).execute(SELECT_QUERY);
testSubject.execute(SELECT_QUERY);
assertWrappedClientIsCalled(2);
assertFallbackIsNotCalled();
assertErrorActionIsCalled(1);
}
@Test
public void testConstantErrorDuringExecution() throws DatabaseException {
doThrow(new DatabaseException("Test")).when(client).execute(SELECT_QUERY);
testSubject.execute(SELECT_QUERY);
assertWrappedClientIsCalled(NUMBER_OF_ATTEMPTS);
assertFallbackIsCalled();
assertErrorActionIsCalled(NUMBER_OF_ATTEMPTS);
}
@Test
public void testWhenTheWriteLockIsOngoing() throws DatabaseException, InterruptedException {
final CountDownLatch countDownLatch = new CountDownLatch(1);
new Thread(() -> {
databaseStructureLock.writeLock().lock();
countDownLatch.countDown();
}).start();
countDownLatch.await();
testSubject.execute(SELECT_QUERY);
assertWrappedClientIsNotCalled();
assertFallbackIsCalled();
assertErrorActionIsCalled(NUMBER_OF_ATTEMPTS);
}
@Test
public void testWhenTheWriteLockIsOngoingButReleased() throws DatabaseException, InterruptedException {
final CountDownLatch lockLatch = new CountDownLatch(1);
final CountDownLatch unlockLatch = new CountDownLatch(1);
final AtomicInteger numberOfCallsExecuted = new AtomicInteger(0);
// In order to be able to release lock from the same thread, we stall the execution until after the first attempt
new Thread(() -> {
databaseStructureLock.writeLock().lock();
lockLatch.countDown();
try {
unlockLatch.await();
} catch (final InterruptedException e) {
throw new RuntimeException(e);
}
databaseStructureLock.writeLock().unlock();
}).start();
// The actual LockedQuestDbClient is surrounded with a spy, which allows the worker thread above to run at the second attempt
doAnswer(invocationOnMock -> {
if (numberOfCallsExecuted.get() == 1) {
unlockLatch.countDown();
}
numberOfCallsExecuted.incrementAndGet();
return invocationOnMock.callRealMethod();
}).when(lockedClientSpy).execute(SELECT_QUERY);
lockLatch.await();
testSubject.execute(SELECT_QUERY);
Assertions.assertEquals(2, numberOfCallsExecuted.get());
assertWrappedClientIsCalled(1);
assertFallbackIsNotCalled();
assertErrorActionIsCalled(1);
}
@Test
public void testWhenConditionDisallowsRun() throws DatabaseException {
when(condition.check()).thenReturn(false);
testSubject.execute(SELECT_QUERY);
assertWrappedClientIsNotCalled();
assertFallbackIsCalled();
assertErrorActionIsCalled(NUMBER_OF_ATTEMPTS);
}
@Test
public void testWhenConditionDisallowsRunForTheFirstAttempt() throws DatabaseException {
when(condition.check()).thenReturn(false).thenReturn(true);
testSubject.execute(SELECT_QUERY);
assertWrappedClientIsCalled(1);
assertFallbackIsNotCalled();
assertErrorActionIsCalled(1);
}
private Client getTestSubject() {
final LockedClient lockedClient = new LockedClient(
databaseStructureLock.readLock(),
LOCK_ATTEMPT_DURATION,
new ConditionAwareClient(condition, client)
);
lockedClientSpy = spy(lockedClient);
return RetryingClient.getInstance(NUMBER_OF_RETRIES, errorAction, lockedClientSpy, fallback);
}
private void assertWrappedClientIsNotCalled() throws DatabaseException {
verify(client, never()).execute(anyString());
}
private void assertWrappedClientIsCalled(final int times) throws DatabaseException {
verify(client, times(times)).execute(SELECT_QUERY);
}
private void assertFallbackIsNotCalled() throws DatabaseException {
verify(fallback, never()).execute(anyString());
}
private void assertFallbackIsCalled() throws DatabaseException {
verify(fallback, times(1)).execute(SELECT_QUERY);
}
private void assertErrorActionIsCalled(final int times) {
verify(errorAction, times(times)).accept(any(Integer.class), any(Exception.class));
}
private void assertErrorActionIsNotCalled() {
verify(errorAction, never()).accept(any(Integer.class), any(Exception.class));
}
}

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.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.mapping.RequestMapping;
import org.apache.nifi.questdb.util.Event;
import org.apache.nifi.questdb.util.QuestDbTestUtil;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import java.util.Collections;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyString;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@ExtendWith(MockitoExtension.class)
public class ConditionAwareClientTest {
@Mock
Client client;
@Test
public void testClientIsCalledWhenConditionAllows() throws DatabaseException {
final ConditionAwareClient testSubject = new ConditionAwareClient(() -> true, client);
testSubject.execute(QuestDbTestUtil.SELECT_QUERY);
verify(client, times(1)).execute(QuestDbTestUtil.SELECT_QUERY);
final QueryResultProcessor<List<Event>> queryResultProcessor = RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING);
testSubject.query(QuestDbTestUtil.SELECT_QUERY, queryResultProcessor);
verify(client, times(1)).query(QuestDbTestUtil.SELECT_QUERY, queryResultProcessor);
final InsertRowDataSource insertRowDataSource = QuestDbTestUtil.getEventTableDataSource(Collections.emptyList());
testSubject.insert(QuestDbTestUtil.EVENT_TABLE_NAME, insertRowDataSource);
verify(client, times(1)).insert(QuestDbTestUtil.EVENT_TABLE_NAME, insertRowDataSource);
}
@Test
public void testClientIsNotCalledWhenConditionDisallows() throws DatabaseException {
final ConditionAwareClient testSubject = new ConditionAwareClient(() -> false, client);
assertThrows(DatabaseException.class, () -> testSubject.execute(QuestDbTestUtil.SELECT_QUERY));
verify(client, never()).execute(anyString());
assertThrows(DatabaseException.class, () -> testSubject.query(QuestDbTestUtil.SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)));
verify(client, never()).query(anyString(), any(QueryResultProcessor.class));
assertThrows(DatabaseException.class, () -> testSubject.insert(QuestDbTestUtil.EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(Collections.emptyList())));
verify(client, never()).insert(anyString(), any(InsertRowDataSource.class));
}
}

View File

@ -0,0 +1,93 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.mapping.RequestMapping;
import org.apache.nifi.questdb.util.Event;
import org.apache.nifi.questdb.util.QuestDbTestUtil;
import org.junit.jupiter.api.Test;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.CREATE_EVENT2_TABLE;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.CREATE_EVENT_TABLE;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.EVENT2_TABLE_NAME;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.EVENT_TABLE_NAME;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING_DIFFERENT_ORDER;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.SELECT_QUERY;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.SELECT_QUERY_2;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class EmbeddedClientTest extends ManagedQuestDbTest {
@Test
public void testInsertAndQuery() throws DatabaseException {
final List<Event> testEvents = QuestDbTestUtil.getTestData();
final Client client = getTestSubject();
client.execute(CREATE_EVENT_TABLE);
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testEvents));
final Iterable<Event> result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertQueryResultMatchesWithInserts(result, testEvents);
}
@Test
public void testInsertAndQueryWhenFieldOrderIsDifferent() throws DatabaseException {
final List<Event> testEvents = QuestDbTestUtil.getTestData();
final Client client = getTestSubject();
client.execute(CREATE_EVENT2_TABLE);
client.insert(EVENT2_TABLE_NAME, QuestDbTestUtil.getEventTableDataSourceWithDifferentOrder(testEvents));
final Iterable<Event> result = client.query(SELECT_QUERY_2, RequestMapping.getResultProcessor(EVENT_TABLE_REQUEST_MAPPING_DIFFERENT_ORDER));
assertQueryResultMatchesWithInserts(result, testEvents);
}
@Test
public void testCannotExecuteOperationAfterDisconnected() throws DatabaseException {
final Client client = getTestSubject();
client.execute(CREATE_EVENT_TABLE);
client.disconnect();
assertThrows(DatabaseException.class, () -> client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)));
assertThrows(DatabaseException.class, () -> client.execute(SELECT_QUERY));
assertThrows(DatabaseException.class, () -> client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(QuestDbTestUtil.getTestData())));
}
private Client getTestSubject() {
return EmbeddedQuestDbTestUtil.getEmbeddedClient(engine);
}
private void assertQueryResultMatchesWithInserts(final Iterable<Event> result, final List<Event> testEvents) {
assertNumberOfEntities(3, result);
final Iterator<Event> iterator = result.iterator();
assertEquals(testEvents.get(0), iterator.next());
assertEquals(testEvents.get(1), iterator.next());
assertEquals(testEvents.get(2), iterator.next());
}
private void assertNumberOfEntities(final int expectedNumber, final Iterable<Event> iterable) {
final AtomicInteger counted = new AtomicInteger(0);
iterable.forEach(e -> counted.incrementAndGet());
assertEquals(expectedNumber, counted.get());
}
}

View File

@ -0,0 +1,410 @@
/*
* 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.questdb.embedded;
import org.apache.commons.lang3.SystemUtils;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.DatabaseManager;
import org.apache.nifi.questdb.mapping.RequestMapping;
import org.apache.nifi.questdb.rollover.RolloverStrategy;
import org.apache.nifi.questdb.util.Event;
import org.apache.nifi.questdb.util.QuestDbTestUtil;
import org.apache.nifi.util.file.FileUtils;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.condition.DisabledOnOs;
import org.junit.jupiter.api.condition.EnabledIfSystemProperty;
import org.junit.jupiter.api.condition.OS;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.BufferedWriter;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.StreamSupport;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.CREATE_EVENT2_TABLE;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.CREATE_EVENT_TABLE;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.EVENT2_TABLE_NAME;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.EVENT_TABLE_NAME;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.SELECT_QUERY;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.getRandomTestData;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.getTestData;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class EmbeddedDatabaseManagerTest extends EmbeddedQuestDbTest {
private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedDatabaseManagerTest.class);
private static final int DAYS_TO_KEEP_EVENT = 1;
private static final String NON_EXISTING_PLACE = SystemUtils.IS_OS_WINDOWS ? "T:/nonExistingPlace" : "/nonExistingPlace";
@Test
public void testAcquiringWithoutInitialization() {
final EmbeddedDatabaseManager testSubject = new EmbeddedDatabaseManager(new SimpleEmbeddedDatabaseManagerContext());
assertThrows(IllegalStateException.class, testSubject::acquireClient);
}
@Test
public void testHappyPath() throws DatabaseException {
final List<Event> testData = getTestData();
final DatabaseManager testSubject = getTestSubject();
assertDatabaseFolderIsNotEmpty();
final Client client = testSubject.acquireClient();
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
final List<Event> result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertIterableEquals(testData, result);
testSubject.close();
// Even if the client itself is not connected, manager prevents client to reach database after closing
assertFalse(client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)).iterator().hasNext());
}
@Test
public void testRollover() throws DatabaseException, InterruptedException {
final List<Event> testData = new ArrayList<>();
testData.add(new Event(Instant.now().minus((DAYS_TO_KEEP_EVENT + 1), ChronoUnit.DAYS), "A", 1));
testData.add(new Event(Instant.now(), "B", 2));
final DatabaseManager testSubject = getTestSubject();
final Client client = testSubject.acquireClient();
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
final long pollTimeoutDuration = 3000;
final long pollTimeout = System.currentTimeMillis() + pollTimeoutDuration;
final int expectedNumberOfResults = 1;
int numberOfResults;
do {
Thread.sleep(100);
final List<Event> result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
numberOfResults = result.size();
} while (numberOfResults != expectedNumberOfResults || pollTimeout > System.currentTimeMillis());
testSubject.close();
assertEquals(expectedNumberOfResults, numberOfResults);
}
@Test
public void testParallelClientsOnSameThread() throws DatabaseException {
final List<Event> testData = getTestData();
final DatabaseManager testSubject = getTestSubject();
final Client client1 = testSubject.acquireClient();
client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
final List<Event> result1 = client1.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
final Client client2 = testSubject.acquireClient();
final List<Event> result2 = client2.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
testSubject.close();
assertEquals(3, result2.size());
assertIterableEquals(result1, result2);
}
@Test
public void testParallelClientsDifferentThread() throws InterruptedException {
final List<Event> testData = getTestData();
final DatabaseManager testSubject = getTestSubject();
final CountDownLatch step1 = new CountDownLatch(1);
final CountDownLatch step2 = new CountDownLatch(1);
final AtomicReference<List<Event>> result1 = new AtomicReference<>();
final AtomicReference<List<Event>> result2 = new AtomicReference<>();
new Thread(() -> {
try {
final Client client1 = testSubject.acquireClient();
client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
result1.set(client1.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)));
step1.countDown();
} catch (final DatabaseException e) {
throw new RuntimeException(e);
}
}).start();
new Thread(() -> {
try {
step1.await();
final Client client2 = testSubject.acquireClient();
result2.set(client2.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)));
step2.countDown();
} catch (final DatabaseException | InterruptedException e) {
throw new RuntimeException(e);
}
}).start();
step2.await();
testSubject.close();
assertEquals(3, result1.get().size());
assertIterableEquals(result1.get(), result2.get());
}
@Test
public void testContactingToDatabaseWithDifferentManager() throws DatabaseException {
final List<Event> testData = getTestData();
final DatabaseManager testSubject1 = getTestSubject();
final Client client1 = testSubject1.acquireClient();
client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
final List<Event> result1 = client1.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
client1.disconnect();
testSubject1.close();
assertDatabaseFolderIsNotEmpty();
final DatabaseManager testSubject2 = getTestSubject();
final Client client2 = testSubject2.acquireClient();
final List<Event> result2 = client2.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
client2.disconnect();
testSubject2.close();
assertIterableEquals(result1, result2);
}
@Test
public void testDatabaseRestorationAfterLostDatabase() throws DatabaseException {
final List<Event> testData = getTestData();
final DatabaseManager testSubject = getTestSubject();
final Client client = testSubject.acquireClient();
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
FileUtils.deleteFilesInDir(testDbPathDirectory.toFile(), (dir, name) -> true, LOGGER, true, true);
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
final List<Event> result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
testSubject.close();
// Ensuring that not the fallback client answers
assertEquals(3, result.size());
}
@Test
public void testDatabaseRestorationAfterLosingTableFiles() throws DatabaseException {
final List<Event> testData = getTestData();
final DatabaseManager testSubject = getTestSubject();
final Client client = testSubject.acquireClient();
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
final File eventTableDirectory = new File(testDbPathDirectory.toFile(), "event");
FileUtils.deleteFilesInDir(eventTableDirectory, (dir, name) -> true, LOGGER, true, true);
FileUtils.deleteFile(eventTableDirectory, LOGGER);
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData));
final List<Event> result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
testSubject.close();
// Ensuring that not the fallback client answers
assertEquals(3, result.size());
}
@Test
@DisabledOnOs(value = OS.WINDOWS, disabledReason = "This testcase cannot be reproduced under Windows using Junit")
public void testDatabaseRestorationAfterCorruptedFiles() throws DatabaseException, IOException {
final DatabaseManager testSubject1 = getTestSubject();
final Client client1 = testSubject1.acquireClient();
for (int i = 1; i <= 10; i++) {
client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getTestData()));
}
final List<Event> result1 = client1.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertEquals(30, result1.size());
client1.disconnect();
testSubject1.close();
corruptDatabaseFile();
// Corrupting the persisted files will not directly affect the database immediately. In order to enforce reading
// information from the files, we need to have a new manager. In real cases this behaviour might be triggered
// during normal usage.
final DatabaseManager testSubject2 = getTestSubject();
final Client client2 = testSubject2.acquireClient();
final List<Event> result2 = client2.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertEquals(0, result2.size());
client2.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getTestData()));
final List<Event> result3 = client2.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
// After successful recreation of the database, the manager does not fall back to NoOp behaviour
assertEquals(3, result3.size());
final File backupDirectory = new File(testDbPathDirectory.toFile().getParentFile(), "questDbBackup");
final List<File> backup = Arrays.asList(backupDirectory.listFiles(file -> file.isDirectory() && file.getName().startsWith("backup_")));
assertFalse(backup.isEmpty());
client2.disconnect();
testSubject2.close();
FileUtils.deleteFile(backupDirectory, true);
}
@Test
@DisabledOnOs(value = OS.WINDOWS, disabledReason = "This testcase cannot be reproduced under Windows using Junit")
public void testWhenBackupIsUnsuccessfulManagerRemovesItAndContinuesWork() throws DatabaseException, IOException {
final DatabaseManager testSubject1 = getTestSubjectBuilder().backupLocation(NON_EXISTING_PLACE).build();
final Client client1 = testSubject1.acquireClient();
for (int i = 1; i <= 10; i++) {
client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getTestData()));
}
final List<Event> result1 = client1.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertEquals(30, result1.size());
corruptDatabaseFile();
client1.disconnect();
testSubject1.close();
final DatabaseManager testSubject2 = getTestSubjectBuilder().backupLocation(NON_EXISTING_PLACE).build();
final Client client2 = testSubject2.acquireClient();
client2.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getTestData()));
final List<Event> result2 = client2.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
// After corrupted database cannot be moved out from the persist folder it is simply deleted
assertEquals(3, result2.size());
// Close Database Manager to avoid orphaned files
testSubject2.close();
}
@Test
@DisabledOnOs(value = OS.WINDOWS, disabledReason = "This testcase cannot be reproduced under Windows using Junit")
public void testWhenRestorationIsUnsuccessfulManagerFallsBackToNoOp() throws DatabaseException, IOException {
final DatabaseManager testSubject1 = getTestSubjectBuilder().backupLocation(NON_EXISTING_PLACE).build();
final Client client1 = testSubject1.acquireClient();
for (int i = 1; i <= 10; i++) {
client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getTestData()));
}
final List<Event> result1 = client1.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertEquals(30, result1.size());
corruptDatabaseFile();
testDbPathDirectory.toFile().setWritable(false);
client1.disconnect();
testSubject1.close();
final DatabaseManager testSubject2 = getTestSubject();
final Client client2 = testSubject2.acquireClient();
client2.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getTestData()));
final List<Event> result2 = client2.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertEquals(0, result2.size());
testDbPathDirectory.toFile().setWritable(true);
client2.disconnect();
testSubject2.close();
final File backupDirectory = new File(testDbPathDirectory.toFile().getParentFile(), "questDbBackup");
FileUtils.deleteFile(backupDirectory, true);
}
@Test
public void testFallsBackToNoOpWhenCannotEnsureDatabaseHealth() throws DatabaseException {
final DatabaseManager testSubject = getTestSubjectBuilder(NON_EXISTING_PLACE).build();
final Client client = testSubject.acquireClient();
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getTestData()));
final List<Event> result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
assertEquals(0, result.size());
client.disconnect();
testSubject.close();
}
/**
* This test case is not part of the normally running test set and needs preparation. In order to successfully
* run this test method, a dedicated partition is necessary with relatively small space. The test intends to
* examine, how the manager behaves when the disk used for persisting the database runs out of space. It is suggested
* to create a "memdisk" for this particular test.
*/
@Test
@Timeout(10)
@EnabledIfSystemProperty(named = "testQuestDBOutOfSpace", matches = "true")
public void testDiskRunOutOfSpace() throws DatabaseException {
final DatabaseManager testSubject = getTestSubjectBuilder("/Volumes/RAM_Disk/testDb").build();
final Client client = testSubject.acquireClient();
boolean reachedBreakdown = false;
while (!reachedBreakdown) {
client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(getRandomTestData()));
final List<Event> result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
if (StreamSupport.stream(result.spliterator(), false).count() == 0) {
reachedBreakdown = true;
}
}
}
private DatabaseManager getTestSubject() {
return getTestSubjectBuilder().build();
}
private EmbeddedDatabaseManagerBuilder getTestSubjectBuilder() {
return getTestSubjectBuilder(testDbPathDirectory.toFile().getAbsolutePath());
}
private static EmbeddedDatabaseManagerBuilder getTestSubjectBuilder(final String persistLocation) {
return EmbeddedDatabaseManagerBuilder
.builder(persistLocation)
.lockAttemptTime(50, TimeUnit.MILLISECONDS)
.rolloverFrequency(1, TimeUnit.SECONDS)
.numberOfAttemptedRetries(2)
.addTable(EVENT_TABLE_NAME, CREATE_EVENT_TABLE, RolloverStrategy.deleteOld(DAYS_TO_KEEP_EVENT))
.addTable(EVENT2_TABLE_NAME, CREATE_EVENT2_TABLE, RolloverStrategy.keep());
}
private void corruptDatabaseFile() throws IOException {
final File fileToCorrupt = new File(new File(testDbPathDirectory.toFile(), "event"), "subject.o");
final FileWriter fileWriter = new FileWriter(fileToCorrupt);
final BufferedWriter bufferedWriter = new BufferedWriter(fileWriter);
bufferedWriter.write("This should corrupt the db");
bufferedWriter.newLine();
bufferedWriter.close();
fileWriter.close();
}
private void assertDatabaseFolderIsNotEmpty() {
final String[] files = testDbPathDirectory.toFile().list();
final int filesFound = files == null ? 0 : files.length;
assertTrue(filesFound > 0);
}
}

View File

@ -0,0 +1,37 @@
/*
* 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.questdb.embedded;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.io.TempDir;
import java.net.URL;
import java.nio.file.Path;
public abstract class EmbeddedQuestDbTest {
@TempDir
protected Path testDbPathDirectory;
@BeforeAll
public static void setLogging() {
final URL logConfUrl = EmbeddedQuestDbTest.class.getResource("/log-stdout.conf");
if (logConfUrl == null) {
throw new IllegalStateException("QuestDB log configuration not found");
}
System.setProperty("out", logConfUrl.getPath());
}
}

View File

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.questdb.embedded;
import io.questdb.cairo.CairoConfiguration;
import io.questdb.cairo.CairoEngine;
import io.questdb.cairo.DefaultCairoConfiguration;
import org.apache.nifi.questdb.Client;
public class EmbeddedQuestDbTestUtil {
public static CairoEngine getEngine(final CharSequence databaseLocation) {
final CairoConfiguration configuration = new DefaultCairoConfiguration(databaseLocation);
return new CairoEngine(configuration);
}
public static Client getEmbeddedClient(final CairoEngine engine) {
return new EmbeddedClient(() -> engine);
}
}

View File

@ -0,0 +1,348 @@
/*
* 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.questdb.embedded;
import org.apache.nifi.questdb.Client;
import org.apache.nifi.questdb.DatabaseException;
import org.apache.nifi.questdb.InsertRowDataSource;
import org.apache.nifi.questdb.QueryResultProcessor;
import org.apache.nifi.questdb.mapping.RequestMapping;
import org.apache.nifi.questdb.util.QuestDbTestUtil;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mockito;
import java.time.Duration;
import java.util.Collections;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.EVENT_TABLE_NAME;
import static org.apache.nifi.questdb.util.QuestDbTestUtil.SELECT_QUERY;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.fail;
public class LockedClientTest {
private static final Duration LOCK_ATTEMPT_DURATION = Duration.of(20, TimeUnit.MILLISECONDS.toChronoUnit());
// The LockedQuestDbClient supports different types of locks as well, but the intention here is to prove expected behaviour with read-write lock
private ReentrantReadWriteLock lock;
private CountDownLatch latchStep1;
private CountDownLatch latchStep2;
private CountDownLatch latchStep3;
@BeforeEach
public void setUp() {
lock = new ReentrantReadWriteLock();
latchStep1 = new CountDownLatch(1);
latchStep2 = new CountDownLatch(1);
latchStep3 = new CountDownLatch(1);
}
@Test
public void testCompile() throws InterruptedException {
final Client testSubject = new Step3SustainerClient(
latchStep3,
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(latchStep1, latchStep2))
);
executeCompileOnDifferentThread(testSubject);
assertLockedOnlyDuringExecution();
}
@Test
public void testCompileWhenException() throws InterruptedException {
final Client testSubject = new Step3SustainerClient(
latchStep3,
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(latchStep1, latchStep2, true))
);
executeCompileOnDifferentThread(testSubject);
assertLockedOnlyDuringExecution();
}
@Test
public void testInsertEntries() throws InterruptedException {
final Client testSubject = new Step3SustainerClient(
latchStep3,
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(latchStep1, latchStep2))
);
executeInsertOnDifferentThread(testSubject);
assertLockedOnlyDuringExecution();
}
@Test
public void testInsertEntriesWhenException() throws InterruptedException {
final Client testSubject = new Step3SustainerClient(
latchStep3,
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(latchStep1, latchStep2, true))
);
executeInsertOnDifferentThread(testSubject);
assertLockedOnlyDuringExecution();
}
@Test
public void testQuery() throws InterruptedException {
final Client testSubject = new Step3SustainerClient(
latchStep3,
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(latchStep1, latchStep2))
);
executeQueryOnDifferentThread(testSubject);
assertLockedOnlyDuringExecution();
}
@Test
public void testQueryWhenException() throws InterruptedException {
final Client testSubject = new Step3SustainerClient(
latchStep3,
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(latchStep1, latchStep2, true))
);
executeQueryOnDifferentThread(testSubject);
assertLockedOnlyDuringExecution();
}
@Test
public void testLockIsReleasedAfterExceptionWhenCompile() {
final Client testSubject = new Step3SustainerClient(
new CountDownLatch(0),
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(new CountDownLatch(0), new CountDownLatch(0), true))
);
assertThrows(ConditionFailedException.class, () -> testSubject.execute(SELECT_QUERY));
assertEquals(0, lock.getReadLockCount());
}
@Test
public void testLockIsReleasedAfterExceptionWhenInsertEntries() {
final Client testSubject = new Step3SustainerClient(
new CountDownLatch(0),
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(new CountDownLatch(0), new CountDownLatch(0), true))
);
assertThrows(ConditionFailedException.class, () -> testSubject.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(Collections.emptyList())));
assertEquals(0, lock.getReadLockCount());
}
@Test
public void testLockIsReleasedAfterExceptionWhenQueryEntries() {
final Client testSubject = new Step3SustainerClient(
new CountDownLatch(0),
new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, new Step1And2SustainerClient(new CountDownLatch(0), new CountDownLatch(0), true))
);
assertThrows(ConditionFailedException.class, () -> testSubject.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)));
assertEquals(0, lock.getReadLockCount());
}
@Test
public void testCompileWhenUnsuccessfulLockingBecauseOfReadLock() throws InterruptedException {
final Client client = Mockito.mock(Client.class);
final Client testSubject = new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, client);
final CountDownLatch lockLatch = new CountDownLatch(1);
new Thread(() -> {
lock.writeLock().lock();
lockLatch.countDown();
}).start();
lockLatch.await();
assertThrows(LockUnsuccessfulException.class, () -> testSubject.execute(SELECT_QUERY));
}
@Test
public void testInsertWhenUnsuccessfulLockingBecauseOfReadLock() throws InterruptedException {
final Client client = Mockito.mock(Client.class);
final Client testSubject = new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, client);
final CountDownLatch lockLatch = new CountDownLatch(1);
new Thread(() -> {
lock.writeLock().lock();
lockLatch.countDown();
}).start();
lockLatch.await();
assertThrows(LockUnsuccessfulException.class, () -> testSubject.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(Collections.emptyList())));
}
@Test
public void testQueryWhenUnsuccessfulLockingBecauseOfReadLock() throws InterruptedException {
final Client client = Mockito.mock(Client.class);
final Client testSubject = new LockedClient(lock.readLock(), LOCK_ATTEMPT_DURATION, client);
final CountDownLatch lockLatch = new CountDownLatch(1);
new Thread(() -> {
lock.writeLock().lock();
lockLatch.countDown();
}).start();
lockLatch.await();
assertThrows(LockUnsuccessfulException.class, () -> testSubject.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)));
}
private static void executeQueryOnDifferentThread(final Client testSubject) {
new Thread(() -> {
try {
testSubject.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING));
} catch (DatabaseException e) {
fail();
}
}).start();
}
private void executeCompileOnDifferentThread(final Client testSubject) {
new Thread(() -> {
try {
testSubject.execute(SELECT_QUERY);
} catch (DatabaseException e) {
fail();
}
}).start();
}
private void executeInsertOnDifferentThread(final Client testSubject) {
new Thread(() -> {
try {
testSubject.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(Collections.emptyList()));
} catch (DatabaseException e) {
fail();
}
}).start();
}
private void assertLockedOnlyDuringExecution() throws InterruptedException {
latchStep1.await();
assertEquals(1, lock.getReadLockCount());
latchStep2.countDown();
latchStep3.await();
assertEquals(0, lock.getReadLockCount());
}
private static class Step1And2SustainerClient implements Client {
private final CountDownLatch step1countDownLatch;
private final CountDownLatch step2countDownLatch;
private final boolean doThrowException;
private Step1And2SustainerClient(final CountDownLatch step1countDownLatch, final CountDownLatch step2countDownLatch) {
this(step1countDownLatch, step2countDownLatch, false);
}
private Step1And2SustainerClient(final CountDownLatch step1countDownLatch, final CountDownLatch step2countDownLatch, final boolean doThrowException) {
this.step1countDownLatch = step1countDownLatch;
this.step2countDownLatch = step2countDownLatch;
this.doThrowException = doThrowException;
}
@Override
public void execute(final String query) throws DatabaseException {
try {
if (doThrowException) {
throw new ConditionFailedException("Test error");
}
} finally {
step1countDownLatch.countDown();
try {
step2countDownLatch.await();
} catch (final InterruptedException e) {
fail();
throw new RuntimeException(e);
}
}
}
@Override
public void insert(final String tableName, final InsertRowDataSource rowSource) throws DatabaseException {
try {
if (doThrowException) {
throw new ConditionFailedException("Test error");
}
} finally {
step1countDownLatch.countDown();
try {
step2countDownLatch.await();
} catch (final InterruptedException e) {
fail();
throw new RuntimeException(e);
}
}
}
@Override
public <T> T query(final String query, final QueryResultProcessor<T> rowProcessor) throws DatabaseException {
try {
if (doThrowException) {
throw new ConditionFailedException("Test error");
}
return rowProcessor.getResult();
} finally {
step1countDownLatch.countDown();
try {
step2countDownLatch.await();
} catch (final InterruptedException e) {
fail();
throw new RuntimeException(e);
}
}
}
@Override
public void disconnect() {}
}
private static class Step3SustainerClient implements Client {
private final CountDownLatch step3countDownLatch;
private final Client client;
private Step3SustainerClient(final CountDownLatch step3countDownLatch, final Client client) {
this.step3countDownLatch = step3countDownLatch;
this.client = client;
}
@Override
public void execute(final String query) throws DatabaseException {
try {
client.execute(query);
} finally {
step3countDownLatch.countDown();
}
}
@Override
public void insert(final String tableName, final InsertRowDataSource rowSource) throws DatabaseException {
try {
client.insert(tableName, rowSource);
} finally {
step3countDownLatch.countDown();
}
}
@Override
public <T> T query(final String query, final QueryResultProcessor<T> rowProcessor) throws DatabaseException {
try {
return client.query(query, rowProcessor);
} finally {
step3countDownLatch.countDown();
}
}
@Override
public void disconnect() {}
}
}

Some files were not shown because too many files have changed in this diff Show More