mirror of https://github.com/apache/nifi.git
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:
parent
4dfbe05641
commit
a519585b02
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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]]
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
});
|
||||
}
|
||||
}
|
|
@ -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";
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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";
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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";
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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";
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
|
@ -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
|
|
@ -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>
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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
|
|
@ -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;
|
||||
}
|
|
@ -393,4 +393,4 @@ public abstract class AbstractStatusHistoryRepositoryTest {
|
|||
assertEquals(41, status.getCollectionCount());
|
||||
assertEquals(42, status.getCollectionMillis());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
|
@ -15,4 +15,4 @@
|
|||
|
||||
writers=stdout
|
||||
w.stdout.class=io.questdb.log.LogConsoleWriter
|
||||
w.stdout.level=ERROR
|
||||
w.stdout.level=CRITICAL
|
|
@ -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>
|
|
@ -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;
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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
Loading…
Reference in New Issue