From a519585b02f37c654638ece4681928fd3abbe355 Mon Sep 17 00:00:00 2001 From: Bence Simon Date: Tue, 12 Dec 2023 10:48:02 -0600 Subject: [PATCH] NIFI-12236 Improved Fault Tolerance in QuestDB Status Repository - Moved QuestDB components to nifi-questdb-bundle This closes #8152 Signed-off-by: David Handermann --- nifi-assembly/pom.xml | 2 +- .../org/apache/nifi/util/NiFiProperties.java | 29 +- .../main/asciidoc/administration-guide.adoc | 10 +- .../EmbeddedQuestDbRolloverHandler.java | 129 ------ ...mbeddedQuestDbStatusHistoryRepository.java | 301 ------------- .../questdb/QuestDbDatabaseManager.java | 179 -------- .../questdb/QuestDbEntityReadingTemplate.java | 62 --- .../questdb/QuestDbEntityWritingTemplate.java | 53 --- .../history/questdb/QuestDbQueries.java | 132 ------ .../questdb/QuestDbReadingTemplate.java | 77 ---- .../questdb/QuestDbStatusSnapshotMapper.java | 44 -- .../questdb/QuestDbWritingTemplate.java | 81 ---- .../BufferedWriterForStatusStorage.java | 50 --- .../storage/ComponentStatusStorage.java | 56 --- .../storage/ProcessorStatusStorage.java | 41 -- .../status/history/storage/StatusStorage.java | 48 -- .../ComponentCounterWritingTemplate.java | 52 --- .../QuestDbComponentStatusStorage.java | 111 ----- .../QuestDbConnectionStatusStorage.java | 61 --- ...QuestDbGarbageCollectionStatusStorage.java | 80 ---- .../questdb/QuestDbNodeStatusStorage.java | 140 ------ .../QuestDbProcessGroupStatusStorage.java | 62 --- .../QuestDbProcessorStatusStorage.java | 167 ------- ...uestDbRemoteProcessGroupStatusStorage.java | 60 --- .../questdb/StorageStatusReadingTemplate.java | 136 ------ .../questdb/StorageStatusWritingTemplate.java | 59 --- .../BufferedWriterForStatusStorageTest.java | 62 --- .../nifi-framework/pom.xml | 1 - .../nifi-framework-bundle/pom.xml | 14 +- .../nifi-questdb-status-history-nar}/pom.xml | 6 +- .../src/main/resources/META-INF/LICENSE | 0 .../src/main/resources/META-INF/NOTICE | 2 +- .../nifi-questdb-status-history}/pom.xml | 10 +- .../questdb/BufferedStatusHistoryStorage.java | 190 ++++++++ .../history/questdb/CapturedStatus.java} | 42 +- .../questdb}/ComponentDetailsStorage.java | 6 +- .../questdb/ComponentStatusDataSource.java | 53 +++ .../questdb/CounterStatisticsDataSource.java | 100 +++++ .../CounterStatisticsResultProcessor.java | 74 ++++ ...mbeddedQuestDbStatusHistoryRepository.java | 243 +++++++++++ ...tDbStatusHistoryRepositoryDefinitions.java | 353 +++++++++++++++ .../GarbageCollectionResultProcessor.java | 49 +++ .../GarbageCollectionStatusDataSource.java | 45 ++ .../InMemoryComponentDetailsStorage.java | 6 +- .../history/questdb/NodeStatusDataSource.java | 50 +++ .../questdb/NodeStatusResultProcessor.java | 70 +++ .../questdb/QuestDbStatusHistoryStorage.java | 181 ++++++++ .../history/questdb/StatusHistoryStorage.java | 50 +++ .../history/questdb/StorageMetric.java} | 32 +- .../questdb/StorageStatusDataSource.java | 87 ++++ .../questdb/StorageStatusResultProcessor.java | 87 ++++ .../history/questdb/StorageStatusType.java | 57 +++ ...ler.status.history.StatusHistoryRepository | 2 +- ...dedQuestDbStatusHistoryRepositoryTest.java | 14 +- .../AbstractStatusHistoryRepositoryTest.java | 2 +- ...tusHistoryRepositoryForComponentsTest.java | 0 ...tDbStatusHistoryRepositoryForNodeTest.java | 1 - .../src/test/resources/log-stdout.conf | 2 +- .../nifi-questdb-bundle/nifi-questdb/pom.xml | 62 +++ .../java/org/apache/nifi/questdb/Client.java | 63 +++ .../nifi/questdb/DatabaseException.java | 32 ++ .../apache/nifi/questdb/DatabaseManager.java} | 30 +- .../nifi/questdb/InsertRowContext.java} | 26 +- .../nifi/questdb/InsertRowDataSource.java | 22 + .../nifi/questdb/QueryResultProcessor.java | 40 ++ .../apache/nifi/questdb/QueryRowContext.java | 25 ++ .../embedded/ClientDisconnectedException.java | 26 ++ .../embedded/ConditionAwareClient.java | 78 ++++ .../embedded/ConditionFailedException.java | 26 ++ .../embedded/CorruptedDatabaseException.java | 33 ++ .../embedded/CursorBasedQueryRowContext.java | 63 +++ .../nifi/questdb/embedded/EmbeddedClient.java | 151 +++++++ .../embedded/EmbeddedDatabaseManager.java | 294 +++++++++++++ .../EmbeddedDatabaseManagerBuilder.java | 97 +++++ .../EmbeddedDatabaseManagerContext.java | 36 ++ .../EmbeddedDatabaseManagerStatus.java | 44 ++ .../embedded/LockUnsuccessfulException.java | 29 ++ .../nifi/questdb/embedded/LockedClient.java | 105 +++++ .../embedded/ManagedTableDefinition.java | 58 +++ .../nifi/questdb/embedded/NoOpClient.java} | 38 +- .../nifi/questdb/embedded/RetryingClient.java | 124 ++++++ .../nifi/questdb/embedded/RolloverWorker.java | 49 +++ .../SimpleEmbeddedDatabaseManagerContext.java | 109 +++++ .../embedded}/SqlExecutionContextFactory.java | 4 +- .../TableWriterBasedInsertRowContext.java | 94 ++++ .../nifi/questdb/mapping/RequestMapping.java | 33 ++ ...questMappingBasedQueryResultProcessor.java | 54 +++ .../mapping/RequestMappingBuilder.java | 67 +++ .../questdb/mapping/SimpleRequestMapping.java | 57 +++ .../org/apache/nifi/questdb/package-info.java | 75 ++++ .../rollover/DeleteOldRolloverStrategy.java | 95 ++++ .../rollover/KeepAllRolloverStrategy.java | 30 ++ .../questdb/rollover/RolloverStrategy.java | 46 ++ .../questdb/embedded/CompositeClientTest.java | 226 ++++++++++ .../embedded/ConditionAwareClientTest.java | 76 ++++ .../questdb/embedded/EmbeddedClientTest.java | 93 ++++ .../embedded/EmbeddedDatabaseManagerTest.java | 410 ++++++++++++++++++ .../questdb/embedded/EmbeddedQuestDbTest.java | 37 ++ .../embedded/EmbeddedQuestDbTestUtil.java | 33 ++ .../questdb/embedded/LockedClientTest.java | 348 +++++++++++++++ .../questdb/embedded/ManagedQuestDbTest.java | 35 ++ .../questdb/embedded/RetryingClientTest.java | 194 +++++++++ .../mapping/RequestMappingBuilderTest.java | 30 ++ .../DeleteOldRolloverStrategyTest.java | 90 ++++ .../org/apache/nifi/questdb/util/Event.java | 85 ++++ .../nifi/questdb/util/QuestDbTestUtil.java | 118 +++++ .../src/test/resources/log-stdout.conf | 18 + .../pom.xml | 9 +- nifi-nar-bundles/pom.xml | 15 + 109 files changed, 5772 insertions(+), 2373 deletions(-) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbRolloverHandler.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepository.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbDatabaseManager.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityReadingTemplate.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityWritingTemplate.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbQueries.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbReadingTemplate.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusSnapshotMapper.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbWritingTemplate.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ComponentStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ProcessorStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/StatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/ComponentCounterWritingTemplate.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbComponentStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbConnectionStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbGarbageCollectionStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbNodeStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessGroupStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessorStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbRemoteProcessGroupStatusStorage.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusReadingTemplate.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusWritingTemplate.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorageTest.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar => nifi-questdb-bundle/nifi-questdb-status-history-nar}/pom.xml (89%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar => nifi-questdb-bundle/nifi-questdb-status-history-nar}/src/main/resources/META-INF/LICENSE (100%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar => nifi-questdb-bundle/nifi-questdb-status-history-nar}/src/main/resources/META-INF/NOTICE (96%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history => nifi-questdb-bundle/nifi-questdb-status-history}/pom.xml (90%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/BufferedStatusHistoryStorage.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbContext.java => nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CapturedStatus.java} (53%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history => nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb}/ComponentDetailsStorage.java (90%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/ComponentStatusDataSource.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsDataSource.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsResultProcessor.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepository.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepositoryDefinitions.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionResultProcessor.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionStatusDataSource.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history => nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb}/InMemoryComponentDetailsStorage.java (88%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusDataSource.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusResultProcessor.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusHistoryStorage.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StatusHistoryStorage.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedEntryWriter.java => nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageMetric.java} (61%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusDataSource.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusResultProcessor.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusType.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history => nifi-questdb-bundle/nifi-questdb-status-history}/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.StatusHistoryRepository (89%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history => nifi-questdb-bundle/nifi-questdb-status-history}/src/test/java/org/apache/nifi/controller/status/history/AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.java (81%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history => nifi-questdb-bundle/nifi-questdb-status-history}/src/test/java/org/apache/nifi/controller/status/history/AbstractStatusHistoryRepositoryTest.java (99%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history => nifi-questdb-bundle/nifi-questdb-status-history}/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForComponentsTest.java (100%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history => nifi-questdb-bundle/nifi-questdb-status-history}/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForNodeTest.java (99%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history => nifi-questdb-bundle/nifi-questdb-status-history}/src/test/resources/log-stdout.conf (97%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/pom.xml create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/Client.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseException.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/NodeStatusStorage.java => nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseManager.java} (56%) rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/GarbageCollectionStatusStorage.java => nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowContext.java} (53%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowDataSource.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryResultProcessor.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryRowContext.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ClientDisconnectedException.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionAwareClient.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionFailedException.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CorruptedDatabaseException.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CursorBasedQueryRowContext.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedClient.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManager.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerBuilder.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerContext.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerStatus.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockUnsuccessfulException.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockedClient.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ManagedTableDefinition.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterFlushWorker.java => nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/NoOpClient.java} (54%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RetryingClient.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RolloverWorker.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/SimpleEmbeddedDatabaseManagerContext.java rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb => nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded}/SqlExecutionContextFactory.java (89%) create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/TableWriterBasedInsertRowContext.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMapping.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBasedQueryResultProcessor.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBuilder.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/SimpleRequestMapping.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/package-info.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategy.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/KeepAllRolloverStrategy.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/RolloverStrategy.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/CompositeClientTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ConditionAwareClientTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedClientTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTestUtil.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/LockedClientTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ManagedQuestDbTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/RetryingClientTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/mapping/RequestMappingBuilderTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategyTest.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/Event.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/QuestDbTestUtil.java create mode 100644 nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/resources/log-stdout.conf rename nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle => nifi-questdb-bundle}/pom.xml (83%) diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index b8a3669cef..66572d413e 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -1165,7 +1165,7 @@ language governing permissions and limitations under the License. --> org.apache.nifi - nifi-framework-questdb-nar + nifi-questdb-status-history-nar 2.0.0-SNAPSHOT nar diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index 41be112c2f..7109600631 100644 --- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -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); } /** diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc index 97ef2e8728..dedae7f2e7 100644 --- a/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -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]] diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbRolloverHandler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbRolloverHandler.java deleted file mode 100644 index 8c524917db..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbRolloverHandler.java +++ /dev/null @@ -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 timeSource; - private final List tables = new ArrayList<>(); - private final int daysToKeepData; - private final QuestDbContext dbContext; - - EmbeddedQuestDbRolloverHandler(final Supplier timeSource, final Collection tables, final int daysToKeepData, final QuestDbContext dbContext) { - this.timeSource = timeSource; - this.tables.addAll(tables); - this.daysToKeepData = daysToKeepData; - this.dbContext = dbContext; - } - - public EmbeddedQuestDbRolloverHandler(final Collection 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 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 getPartitions(final CharSequence tableName) throws Exception { - final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext(); - final List 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); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepository.java deleted file mode 100644 index c478191924..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepository.java +++ /dev/null @@ -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 connectionStatusStorage; - private final ComponentStatusStorage processGroupStatusStorage; - private final ComponentStatusStorage remoteProcessGroupStatusStorage; - private final NodeStatusStorage nodeStatusStorage; - private final GarbageCollectionStatusStorage garbageCollectionStatusStorage; - - private final BufferedWriterForStatusStorage processorStatusWriter; - private final BufferedWriterForStatusStorage connectionStatusWriter; - private final BufferedWriterForStatusStorage processGroupStatusWriter; - private final BufferedWriterForStatusStorage remoteProcessGroupStatusWriter; - private final BufferedWriterForStatusStorage nodeStatusWriter; - private final BufferedWriterForStatusStorage garbageCollectionStatusWriter; - - private final List> 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 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, - 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, 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 accumulator = new HashMap<>(); - updateComponentDetails(groupStatus, accumulator); - componentDetailsProvider.setComponentDetails(accumulator); - } - - private void updateComponentDetails(final ProcessGroupStatus groupStatus, final Map 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(); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbDatabaseManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbDatabaseManager.java deleted file mode 100644 index 4fe83169df..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbDatabaseManager.java +++ /dev/null @@ -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 COMPONENT_TABLES = new HashSet<>(); - private static final Set 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 databaseFiles = Arrays.stream(persistLocationDirectory.listFiles()) - .collect(Collectors.toMap(f -> f.getAbsolutePath().substring(persistLocationDirectory.getAbsolutePath().length() + 1), f -> f)); - - final Set 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 getNodeTableNames() { - return NODE_TABLES; - } - - public static Set getComponentTableNames() { - return COMPONENT_TABLES; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityReadingTemplate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityReadingTemplate.java deleted file mode 100644 index 2c2c827561..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityReadingTemplate.java +++ /dev/null @@ -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 The entity type represented by a single database record. - * @param The result of the selection. Might be an aggregated value or collection. - */ -public class QuestDbEntityReadingTemplate extends QuestDbReadingTemplate { - private final Function mapper; - private final Function, 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 mapper, - final Function, R> aggregator, - final Function errorResult) { - super(query, errorResult); - this.mapper = mapper; - this.aggregator = aggregator; - } - - @Override - protected R processResult(final RecordCursor cursor) { - final List entities = new LinkedList<>(); - - while (cursor.hasNext()) { - entities.add(mapper.apply(cursor.getRecord())); - } - - return aggregator.apply(entities); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityWritingTemplate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityWritingTemplate.java deleted file mode 100644 index 51ac48c14e..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbEntityWritingTemplate.java +++ /dev/null @@ -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 Entry type. - */ -public class QuestDbEntityWritingTemplate extends QuestDbWritingTemplate> { - private final BiConsumer 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 fillRow) { - super(tableName); - this.fillRow = fillRow; - } - - @Override - protected void addRows(final TableWriter tableWriter, final Collection> 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(); - }); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbQueries.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbQueries.java deleted file mode 100644 index 14986012b0..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbQueries.java +++ /dev/null @@ -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"; -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbReadingTemplate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbReadingTemplate.java deleted file mode 100644 index 48796f0cc0..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbReadingTemplate.java +++ /dev/null @@ -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 The result of the selection. Might be an aggregated value or collection. - */ -abstract public class QuestDbReadingTemplate { - private static final Logger LOGGER = LoggerFactory.getLogger(QuestDbReadingTemplate.class); - - private final String query; - private final Function 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 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 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); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusSnapshotMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusSnapshotMapper.java deleted file mode 100644 index 468476db63..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusSnapshotMapper.java +++ /dev/null @@ -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 implements Function { - private final Map> metrics = new HashMap<>(); - - public QuestDbStatusSnapshotMapper(final Map> 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; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbWritingTemplate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbWritingTemplate.java deleted file mode 100644 index c93fbc59f8..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbWritingTemplate.java +++ /dev/null @@ -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 The type of the entry. - */ -public abstract class QuestDbWritingTemplate { - 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 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 entries); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorage.java deleted file mode 100644 index bea55a525d..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorage.java +++ /dev/null @@ -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 implements BufferedEntryWriter> { - private final BlockingQueue> queue = new LinkedBlockingQueue<>(); - private final StatusStorage statusStorage; - private final int batchSize; - - public BufferedWriterForStatusStorage(final StatusStorage statusStorage, final int batchSize) { - this.statusStorage = statusStorage; - this.batchSize = batchSize; - } - - @Override - public void collect(final Pair entryToStore) { - queue.add(entryToStore); - } - - @Override - public void flush() { - final ArrayList> entries = new ArrayList<>(batchSize); - queue.drainTo(entries, batchSize); - - if (!entries.isEmpty()) { - statusStorage.store(entries); - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ComponentStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ComponentStatusStorage.java deleted file mode 100644 index 173af3cebc..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ComponentStatusStorage.java +++ /dev/null @@ -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 Stored component status entry type. - */ -public interface ComponentStatusStorage extends StatusStorage { - - /** - * 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); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ProcessorStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ProcessorStatusStorage.java deleted file mode 100644 index e7cc407293..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/ProcessorStatusStorage.java +++ /dev/null @@ -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 { - /** - * 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); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/StatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/StatusStorage.java deleted file mode 100644 index 44a3e4ce9d..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/StatusStorage.java +++ /dev/null @@ -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 The entry type. - */ -public interface StatusStorage { - /** - * 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> statusEntries); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/ComponentCounterWritingTemplate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/ComponentCounterWritingTemplate.java deleted file mode 100644 index 67da6d986b..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/ComponentCounterWritingTemplate.java +++ /dev/null @@ -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> { - - public ComponentCounterWritingTemplate() { - super("componentCounter"); - } - - @Override - protected void addRows(final TableWriter tableWriter, final Collection> entries) { - for (final Pair entry : entries) { - final Map counters = entry.getRight().getCounters(); - - if (counters != null && counters.size() > 0) { - for (final Map.Entry 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(); - } - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbComponentStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbComponentStatusStorage.java deleted file mode 100644 index a9c79afa73..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbComponentStatusStorage.java +++ /dev/null @@ -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 Component status entry type. - */ -abstract class QuestDbComponentStatusStorage implements ComponentStatusStorage { - - /** - * 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 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 statusSnapshotMapper = new QuestDbStatusSnapshotMapper(getMetrics()); - - private final QuestDbEntityReadingTemplate> 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> 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 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> statusEntries) { - writingTemplate.insert(dbContext.getEngine(), dbContext.getSqlExecutionContext(), statusEntries); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbConnectionStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbConnectionStatusStorage.java deleted file mode 100644 index fcc565547d..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbConnectionStatusStorage.java +++ /dev/null @@ -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 { - private static final Map> 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> getMetrics() { - return METRICS; - } - - @Override - protected String getTableName() { - return "connectionStatus"; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbGarbageCollectionStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbGarbageCollectionStatusStorage.java deleted file mode 100644 index 9b917e6811..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbGarbageCollectionStatusStorage.java +++ /dev/null @@ -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 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 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> statusEntries) { - WRITING_TEMPLATE.insert(context.getEngine(), context.getSqlExecutionContext(), statusEntries); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbNodeStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbNodeStatusStorage.java deleted file mode 100644 index 8fe1f12f1e..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbNodeStatusStorage.java +++ /dev/null @@ -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> 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 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> 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>> 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 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> statusEntries) { - final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext(); - WRITING_TEMPLATE.insert(dbContext.getEngine(), executionContext, statusEntries); - STORAGE_WRITING_TEMPLATE.insert(dbContext.getEngine(), executionContext, statusEntries); - } - - public static Map> getMetrics() { - return METRICS; - } - - private static class NodeStatusReadingTemplate extends QuestDbReadingTemplate> { - private final Map, Long>> storageMetricsByTime; - - public NodeStatusReadingTemplate(final Map, Long>> storageMetricsByTime) { - super(READING_QUERY, e -> Collections.emptyList()); - this.storageMetricsByTime = storageMetricsByTime; - } - - @Override - protected List processResult(final RecordCursor cursor) { - final List 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, Long> statusMetrics = storageMetricsByTime.get(createdAt); - final Set> 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; - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessGroupStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessGroupStatusStorage.java deleted file mode 100644 index 601f2af395..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessGroupStatusStorage.java +++ /dev/null @@ -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 { - private static final Map> 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> getMetrics() { - return METRICS; - } - - @Override - protected String getTableName() { - return "processGroupStatus"; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessorStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessorStatusStorage.java deleted file mode 100644 index c9ded7a99b..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbProcessorStatusStorage.java +++ /dev/null @@ -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> 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 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> counterWritingTemplate = new ComponentCounterWritingTemplate(); - - private final Function statusSnapshotMapper = new QuestDbStatusSnapshotMapper(METRICS); - - private final QuestDbEntityReadingTemplate> 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 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 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 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> statusEntries) { - final SqlExecutionContext executionContext = dbContext.getSqlExecutionContext(); - writingTemplate.insert(dbContext.getEngine(), executionContext, statusEntries); - counterWritingTemplate.insert(dbContext.getEngine(), executionContext, statusEntries); - } - - private final class CounterReadingTemplate extends QuestDbReadingTemplate> { - private final List processorStatusSnapshots; - - public CounterReadingTemplate(final List processorStatusSnapshots) { - super(QUERY_TEMPLATE, e -> Collections.emptyList()); - this.processorStatusSnapshots = processorStatusSnapshots; - } - - @Override - protected List processResult(final RecordCursor cursor) { - final Map 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 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; - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbRemoteProcessGroupStatusStorage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbRemoteProcessGroupStatusStorage.java deleted file mode 100644 index f6511f139f..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/QuestDbRemoteProcessGroupStatusStorage.java +++ /dev/null @@ -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 { - private static final Map> 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> getMetrics() { - return METRICS; - } - - @Override - protected String getTableName() { - return "remoteProcessGroupStatus"; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusReadingTemplate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusReadingTemplate.java deleted file mode 100644 index 58d82b40f6..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusReadingTemplate.java +++ /dev/null @@ -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, 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>> processResult(final RecordCursor cursor) { - final Map, 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 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; - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusWritingTemplate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusWritingTemplate.java deleted file mode 100644 index 55f96d697a..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/questdb/StorageStatusWritingTemplate.java +++ /dev/null @@ -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> { - - public StorageStatusWritingTemplate() { - super("storageStatus"); - } - - @Override - protected void addRows(final TableWriter tableWriter, final Collection> entries) { - for (final Pair 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(); - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorageTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorageTest.java deleted file mode 100644 index f3e9111ded..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/storage/BufferedWriterForStatusStorageTest.java +++ /dev/null @@ -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 payload; - - @Test - public void testStoringOnlyWhenPersist() { - // given - final BufferedWriterForStatusStorage testSubject = new BufferedWriterForStatusStorage<>(payload, BUFFER_SIZE); - final ArgumentCaptor 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()); - } -} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml index 54c3867f71..85af1647c8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml @@ -38,7 +38,6 @@ nifi-framework-leader-election-shared nifi-framework-zookeeper-leader-election nifi-framework-kubernetes-bundle - nifi-framework-questdb-bundle nifi-framework-status-history-shared nifi-framework-nar-utils nifi-framework-nar-loading-utils diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml index 6715bd242c..215d499c7a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml @@ -35,19 +35,7 @@ nifi-headless-server-nar nifi-framework-external-resource-utils - - - - shibboleth - https://build.shibboleth.net/nexus/content/repositories/releases/ - - true - - - false - - - + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/pom.xml b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/pom.xml similarity index 89% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/pom.xml rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/pom.xml index b99e41040e..f52c706910 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/pom.xml +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/pom.xml @@ -16,10 +16,10 @@ 4.0.0 org.apache.nifi - nifi-framework-questdb-bundle + nifi-questdb-bundle 2.0.0-SNAPSHOT - nifi-framework-questdb-nar + nifi-questdb-status-history-nar nar @@ -30,7 +30,7 @@ org.apache.nifi - nifi-framework-questdb-status-history + nifi-questdb-status-history 2.0.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/src/main/resources/META-INF/LICENSE similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/src/main/resources/META-INF/LICENSE rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/src/main/resources/META-INF/LICENSE diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/src/main/resources/META-INF/NOTICE similarity index 96% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/src/main/resources/META-INF/NOTICE rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/src/main/resources/META-INF/NOTICE index 20ca0c226f..13856d212a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-nar/src/main/resources/META-INF/NOTICE +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history-nar/src/main/resources/META-INF/NOTICE @@ -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 diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/pom.xml b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/pom.xml similarity index 90% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/pom.xml rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/pom.xml index 72ccb55f37..0ddbd92b18 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/pom.xml +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/pom.xml @@ -17,10 +17,10 @@ 4.0.0 org.apache.nifi - nifi-framework-questdb-bundle + nifi-questdb-bundle 2.0.0-SNAPSHOT - nifi-framework-questdb-status-history + nifi-questdb-status-history org.apache.nifi @@ -52,9 +52,9 @@ commons-lang3 - org.questdb - questdb - 7.3.7 + org.apache.nifi + nifi-questdb + 2.0.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/BufferedStatusHistoryStorage.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/BufferedStatusHistoryStorage.java new file mode 100644 index 0000000000..27f288823e --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/BufferedStatusHistoryStorage.java @@ -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> 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> nodeStatusQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue> garbageCollectionStatusQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue> processGroupStatusQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue> connectionStatusQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue> remoteProcessGroupStatusQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue> 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 tasks = scheduledExecutorService.shutdownNow(); + LOGGER.debug("Scheduled Task Service shutdown remaining tasks [{}]", tasks.size()); + + } + + @Override + public List getConnectionSnapshots(final String componentId, final Date start, final Date end) { + return storage.getConnectionSnapshots(componentId, start, end); + } + + @Override + public List getProcessGroupSnapshots(final String componentId, final Date start, final Date end) { + return storage.getProcessGroupSnapshots(componentId, start, end); + } + + @Override + public List getRemoteProcessGroupSnapshots(final String componentId, final Date start, final Date end) { + return storage.getRemoteProcessGroupSnapshots(componentId, start, end); + } + + @Override + public List getProcessorSnapshots(final String componentId, final Date start, final Date end) { + return storage.getProcessorSnapshots(componentId, start, end); + } + + @Override + public List getProcessorSnapshotsWithCounters(final String componentId, final Date start, final Date end) { + return storage.getProcessorSnapshotsWithCounters(componentId, start, end); + } + + @Override + public List getGarbageCollectionSnapshots(final Date start, final Date end) { + return storage.getGarbageCollectionSnapshots(start, end); + } + + @Override + public List getNodeStatusSnapshots(final Date start, final Date end) { + return storage.getNodeStatusSnapshots(start, end); + } + + @Override + public void storeNodeStatuses(final Collection> statuses) { + nodeStatusQueue.addAll(statuses); + } + + @Override + public void storeGarbageCollectionStatuses(final Collection> statuses) { + garbageCollectionStatusQueue.addAll(statuses); + } + + @Override + public void storeProcessGroupStatuses(final Collection> statuses) { + processGroupStatusQueue.addAll(statuses); + } + + @Override + public void storeConnectionStatuses(final Collection> statuses) { + connectionStatusQueue.addAll(statuses); + } + + @Override + public void storeRemoteProcessorGroupStatuses(final Collection> statuses) { + remoteProcessGroupStatusQueue.addAll(statuses); + } + + @Override + public void storeProcessorStatuses(final Collection> 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 void flush(final BlockingQueue source, final Consumer> target) { + final ArrayList 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); + } + } + } + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbContext.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CapturedStatus.java similarity index 53% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbContext.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CapturedStatus.java index 0816645d27..de7cb538b6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbContext.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CapturedStatus.java @@ -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 { + 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); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/ComponentDetailsStorage.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/ComponentDetailsStorage.java similarity index 90% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/ComponentDetailsStorage.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/ComponentDetailsStorage.java index 8cd5791ee1..18be332e54 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/ComponentDetailsStorage.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/ComponentDetailsStorage.java @@ -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 diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/ComponentStatusDataSource.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/ComponentStatusDataSource.java new file mode 100644 index 0000000000..79f3432efd --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/ComponentStatusDataSource.java @@ -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 implements InsertRowDataSource { + private final Iterator> statuses; + private final Map> metricDescriptors; + private final Function acquireId; + + ComponentStatusDataSource(final Iterator> statuses, final Map> metricDescriptors, final Function 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 status = statuses.next(); + context.initializeRow(status.getCaptured()); + context.addString(1, acquireId.apply(status.getStatus())); + + for (final Map.Entry> metric : metricDescriptors.entrySet()) { + context.addLong(metric.getKey(), metric.getValue().getValueFunction().getValue(status.getStatus())); + } + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsDataSource.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsDataSource.java new file mode 100644 index 0000000000..56b9d4df16 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsDataSource.java @@ -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 counterStatistics; + + public CounterStatisticsDataSource(final Iterator 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> processorStatuses) { + final List counterStatistics = new LinkedList<>(); + + for (final CapturedStatus processorStatus : processorStatuses) { + if (processorStatus.getStatus().getCounters() != null) { + for (final Map.Entry 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; + } + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsResultProcessor.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsResultProcessor.java new file mode 100644 index 0000000000..62cdecaf54 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/CounterStatisticsResultProcessor.java @@ -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> { + private static final Logger LOGGER = LoggerFactory.getLogger(CounterStatisticsResultProcessor.class); + + private final List processorSnapshots; + final Map processorSnapshotsByTime; + + CounterStatisticsResultProcessor(final List 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 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 getResult() { + return processorSnapshots; + } + + private static CounterMetricDescriptor 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) + ); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepository.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepository.java new file mode 100644 index 0000000000..beec0cbcb7 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepository.java @@ -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, 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 statuses, final Instant captured) { + final Set> 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> wrapConnectionStatuses(final ProcessGroupStatus groupStatus, final Instant captured) { + final Collection statuses = groupStatus.getConnectionStatus(); + final Set> result = new HashSet<>(statuses.size()); + statuses.forEach(status -> result.add(new CapturedStatus<>(status, captured))); + return result; + } + + private Collection> wrapRemoteProcessGroupStatuses(final ProcessGroupStatus groupStatus, final Instant captured) { + final Collection statuses = groupStatus.getRemoteProcessGroupStatus(); + final Set> result = new HashSet<>(statuses.size()); + statuses.forEach(status -> result.add(new CapturedStatus<>(status, captured))); + return result; + } + + private Collection> wrapProcessorStatuses(final ProcessGroupStatus groupStatus, final Instant captured) { + final Collection statuses = groupStatus.getProcessorStatus(); + final Set> 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 snapshots = storage.getGarbageCollectionSnapshots(start, end); + final StandardGarbageCollectionHistory result = new StandardGarbageCollectionHistory(); + snapshots.forEach(result::addGarbageCollectionStatus); + return result; + } + + private StatusHistory generateStatusHistory(final String componentId, final List 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 accumulator = new HashMap<>(); + updateComponentDetails(groupStatus, accumulator); + componentDetailsProvider.setComponentDetails(accumulator); + } + + private void updateComponentDetails(final ProcessGroupStatus groupStatus, final Map 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)); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepositoryDefinitions.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepositoryDefinitions.java new file mode 100644 index 0000000000..3fef39044d --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/EmbeddedQuestDbStatusHistoryRepositoryDefinitions.java @@ -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> 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> statuses) { + return new ComponentStatusDataSource<>(statuses.iterator(), CONNECTION_METRICS, ConnectionStatus::getId); + } + + static final RequestMapping 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> 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> statuses) { + return new ComponentStatusDataSource<>(statuses.iterator(), PROCESSOR_METRICS, ProcessorStatus::getId); + } + + static InsertRowDataSource getCounterStatisticsDataSource(final Collection> statuses) { + return CounterStatisticsDataSource.getInstance(statuses); + } + + static final RequestMapping 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> 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> statuses) { + return new ComponentStatusDataSource<>(statuses.iterator(), PROCESS_GROUP_METRICS, ProcessGroupStatus::getId); + } + + static final RequestMapping 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> 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> statuses) { + return new ComponentStatusDataSource<>(statuses.iterator(), REMOTE_PROCESS_GROUP_METRICS, RemoteProcessGroupStatus::getId); + } + + static final RequestMapping 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> 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, 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> 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> statuses) { + return new NodeStatusDataSource(statuses.iterator(), NODE_STATUS_METRICS); + } + + static QueryResultProcessor> getNodeStatusResultProcessor( + final Map, Long>> statusMetricsByTime + ) { + return new NodeStatusResultProcessor(NODE_STATUS_METRICS, statusMetricsByTime); + } + + private static RequestMapping getSnapshotRequestMapping(Class type, Collection> descriptorSource) { + final RequestMappingBuilder 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 + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionResultProcessor.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionResultProcessor.java new file mode 100644 index 0000000000..75e6097c1c --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionResultProcessor.java @@ -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> { + private final List 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 getResult() { + return result; + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionStatusDataSource.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionStatusDataSource.java new file mode 100644 index 0000000000..d0fc0f5e58 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/GarbageCollectionStatusDataSource.java @@ -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> statuses; + + GarbageCollectionStatusDataSource(final Iterator> statuses) { + this.statuses = statuses; + } + + @Override + public boolean hasNextToInsert() { + return statuses.hasNext(); + } + + @Override + public void fillRowData(final InsertRowContext context) { + final CapturedStatus status = statuses.next(); + context.initializeRow(status.getCaptured()) + .addString(1, status.getStatus().getMemoryManagerName()) + .addLong(2, status.getStatus().getCollectionCount()) + .addLong(3, status.getStatus().getCollectionMillis()); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/InMemoryComponentDetailsStorage.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/InMemoryComponentDetailsStorage.java similarity index 88% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/InMemoryComponentDetailsStorage.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/InMemoryComponentDetailsStorage.java index 4deb2873ef..aabc14c307 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/InMemoryComponentDetailsStorage.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/InMemoryComponentDetailsStorage.java @@ -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> componentDetails = new AtomicReference<>(new HashMap<>()); @Override diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusDataSource.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusDataSource.java new file mode 100644 index 0000000000..feb5be4b2e --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusDataSource.java @@ -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> statuses; + private final Map> metrics; + + NodeStatusDataSource(final Iterator> statuses, final Map> metrics) { + this.statuses = statuses; + this.metrics = metrics; + } + + @Override + public boolean hasNextToInsert() { + return statuses.hasNext(); + } + + @Override + public void fillRowData(final InsertRowContext context) { + final CapturedStatus status = statuses.next(); + context.initializeRow(status.getCaptured()); + + for (final Map.Entry> metric : metrics.entrySet()) { + context.addLong(metric.getKey(), metric.getValue().getValueFunction().getValue(status.getStatus())); + } + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusResultProcessor.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusResultProcessor.java new file mode 100644 index 0000000000..77c8085377 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/NodeStatusResultProcessor.java @@ -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> { + private final Map> metricDescriptors; + private final Map, Long>> storageMetricsByTime; + private final List result = new ArrayList<>(); + + NodeStatusResultProcessor( + final Map> metricDescriptors, + final Map, 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, Long> storageMetrics = storageMetricsByTime.get(createdAt); + final Set> 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> 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 getResult() { + return result; + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusHistoryStorage.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusHistoryStorage.java new file mode 100644 index 0000000000..7baabba146 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/QuestDbStatusHistoryStorage.java @@ -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 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 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 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 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 getProcessorSnapshotsWithCounters(final String componentId, final Date start, final Date end) { + final List 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 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 getNodeStatusSnapshots(final Date start, final Date end) { + final String storageStatusQuery = String.format(STORAGE_STATUS_QUERY, getStartTime(start), getEndTime(end)); + final Map, 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> statuses) { + store(TABLE_NAME_NODE_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getNodeStatusDataSource(statuses)); + store(TABLE_NAME_STORAGE_STATUS, StorageStatusDataSource.getInstance(statuses)); + } + + @Override + public void storeGarbageCollectionStatuses(final Collection> statuses) { + store(TABLE_NAME_GARBAGE_COLLECTION_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getGarbageCollectionStatusDataSource(statuses)); + } + + @Override + public void storeProcessGroupStatuses(final Collection> statuses) { + store(TABLE_NAME_PROCESS_GROUP_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getProcessGroupStatusDataSource(statuses)); + } + + @Override + public void storeConnectionStatuses(final Collection> statuses) { + store(TABLE_NAME_CONNECTION_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getConnectionStatusDataSource(statuses)); + } + + @Override + public void storeRemoteProcessorGroupStatuses(final Collection> statuses) { + store(TABLE_NAME_REMOTE_PROCESS_GROUP_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getRemoteProcessGroupStatusDataSource(statuses)); + } + + @Override + public void storeProcessorStatuses(final Collection> statuses) { + store(TABLE_NAME_PROCESSOR_STATUS, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getProcessorStatusDataSource(statuses)); + store(TABLE_NAME_COMPONENT_COUNTER, EmbeddedQuestDbStatusHistoryRepositoryDefinitions.getCounterStatisticsDataSource(statuses)); + } + + private 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 getComponentSnapshots(final String tableName, final String componentId, final RequestMapping 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 getSnapshot(final String query, final QueryResultProcessor> rowProcessor) { + return new ArrayList<>(getResult(query, rowProcessor, Collections.emptyList())); + } + + private T getResult(final String query, final QueryResultProcessor 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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StatusHistoryStorage.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StatusHistoryStorage.java new file mode 100644 index 0000000000..df4c1b46e9 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StatusHistoryStorage.java @@ -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 getConnectionSnapshots(final String componentId, final Date start, final Date end); + List getProcessGroupSnapshots(final String componentId, final Date start, final Date end); + List getRemoteProcessGroupSnapshots(final String componentId, final Date start, final Date end); + List getProcessorSnapshots(final String componentId, final Date start, final Date end); + List getProcessorSnapshotsWithCounters(final String componentId, final Date start, final Date end); + List getNodeStatusSnapshots(final Date start, final Date end); + List getGarbageCollectionSnapshots(final Date start, final Date end); + + void storeNodeStatuses(final Collection> statuses); + void storeGarbageCollectionStatuses(final Collection> statuses); + void storeProcessGroupStatuses(final Collection> statuses); + void storeConnectionStatuses(final Collection> statuses); + void storeRemoteProcessorGroupStatuses(final Collection> statuses); + void storeProcessorStatuses(final Collection> statuses); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedEntryWriter.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageMetric.java similarity index 61% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedEntryWriter.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageMetric.java index 8fbd486678..a94ac11e6b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedEntryWriter.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageMetric.java @@ -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 { +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; + } } diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusDataSource.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusDataSource.java new file mode 100644 index 0000000000..15a0732720 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusDataSource.java @@ -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 statuses; + + private StorageStatusDataSource(final Iterator 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> nodeStatuses) { + final List statuses = new ArrayList<>(); + for (final CapturedStatus 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; + } + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusResultProcessor.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusResultProcessor.java new file mode 100644 index 0000000000..3287ddb09c --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusResultProcessor.java @@ -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, 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> metricDescriptors; + private final Map, Long>> result = new HashMap<>(); + private int storageNumber = 1; + + StorageStatusResultProcessor(final Map> 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>> getResult() { + return result; + } + + private StandardMetricDescriptor 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(); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusType.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusType.java new file mode 100644 index 0000000000..1d3403b803 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/StorageStatusType.java @@ -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 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; + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.StatusHistoryRepository b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.StatusHistoryRepository similarity index 89% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.StatusHistoryRepository rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.StatusHistoryRepository index aaf2fe5c8c..93ede78fd5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.StatusHistoryRepository +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/main/resources/META-INF/services/org.apache.nifi.controller.status.history.StatusHistoryRepository @@ -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 diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.java similarity index 81% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.java index efb1154d1a..b1ab2278bb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractEmbeddedQuestDbStatusHistoryRepositoryTest.java @@ -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; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractStatusHistoryRepositoryTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractStatusHistoryRepositoryTest.java similarity index 99% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractStatusHistoryRepositoryTest.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractStatusHistoryRepositoryTest.java index 295b5c3742..2e815c70bd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractStatusHistoryRepositoryTest.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/AbstractStatusHistoryRepositoryTest.java @@ -393,4 +393,4 @@ public abstract class AbstractStatusHistoryRepositoryTest { assertEquals(41, status.getCollectionCount()); assertEquals(42, status.getCollectionMillis()); } -} +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForComponentsTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForComponentsTest.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForComponentsTest.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForComponentsTest.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForNodeTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForNodeTest.java similarity index 99% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForNodeTest.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForNodeTest.java index c7da8b2811..dae0bab514 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForNodeTest.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/java/org/apache/nifi/controller/status/history/EmbeddedQuestDbStatusHistoryRepositoryForNodeTest.java @@ -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 diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/resources/log-stdout.conf b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/resources/log-stdout.conf similarity index 97% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/resources/log-stdout.conf rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/resources/log-stdout.conf index 3898586318..4fb020ec29 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/test/resources/log-stdout.conf +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb-status-history/src/test/resources/log-stdout.conf @@ -15,4 +15,4 @@ writers=stdout w.stdout.class=io.questdb.log.LogConsoleWriter -w.stdout.level=ERROR +w.stdout.level=CRITICAL diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/pom.xml b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/pom.xml new file mode 100644 index 0000000000..9dbea7f52b --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/pom.xml @@ -0,0 +1,62 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-questdb-bundle + 2.0.0-SNAPSHOT + + nifi-questdb + + + + org.apache.nifi + nifi-utils + 2.0.0-SNAPSHOT + + + + org.apache.commons + commons-lang3 + + + org.slf4j + slf4j-api + + + org.questdb + questdb + 7.3.7 + + + org.mockito + mockito-junit-jupiter + + + + + org.springframework + spring-core + + + + org.springframework.retry + spring-retry + 2.0.5 + + + diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/Client.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/Client.java new file mode 100644 index 0000000000..7754f36f8f --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/Client.java @@ -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 Result type. + */ + T query( + String query, + QueryResultProcessor rowProcessor + ) throws DatabaseException; + + /** + * Terminates the client. After {@code disconnect} is called, answer for other calls is not guaranteed. + */ + void disconnect() throws DatabaseException; +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseException.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseException.java new file mode 100644 index 0000000000..b4b91e6f90 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseException.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/NodeStatusStorage.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseManager.java similarity index 56% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/NodeStatusStorage.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseManager.java index 63efd110c0..ef2124b7fc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/NodeStatusStorage.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/DatabaseManager.java @@ -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 { +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(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/GarbageCollectionStatusStorage.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowContext.java similarity index 53% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/GarbageCollectionStatusStorage.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowContext.java index 4060525b07..93faeac0ac 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/GarbageCollectionStatusStorage.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowContext.java @@ -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 { +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); } diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowDataSource.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowDataSource.java new file mode 100644 index 0000000000..79d76fb254 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/InsertRowDataSource.java @@ -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); +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryResultProcessor.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryResultProcessor.java new file mode 100644 index 0000000000..6c788c388b --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryResultProcessor.java @@ -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 { + Map, BiFunction> 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(); +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryRowContext.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryRowContext.java new file mode 100644 index 0000000000..566b2c19bd --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/QueryRowContext.java @@ -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); +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ClientDisconnectedException.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ClientDisconnectedException.java new file mode 100644 index 0000000000..f8dc4f4bcc --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ClientDisconnectedException.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionAwareClient.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionAwareClient.java new file mode 100644 index 0000000000..7c64b23598 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionAwareClient.java @@ -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 query(final String query, final QueryResultProcessor 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 + + '}'; + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionFailedException.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionFailedException.java new file mode 100644 index 0000000000..64e9960dd0 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ConditionFailedException.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CorruptedDatabaseException.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CorruptedDatabaseException.java new file mode 100644 index 0000000000..e0735ad9c4 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CorruptedDatabaseException.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CursorBasedQueryRowContext.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CursorBasedQueryRowContext.java new file mode 100644 index 0000000000..2c01cb840c --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/CursorBasedQueryRowContext.java @@ -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(); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedClient.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedClient.java new file mode 100644 index 0000000000..807a42249f --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedClient.java @@ -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 engine; + private final AtomicBoolean disconnected = new AtomicBoolean(false); + + EmbeddedClient(final Supplier 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 query(final String query, final QueryResultProcessor 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()); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManager.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManager.java new file mode 100644 index 0000000000..c71fa41c31 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManager.java @@ -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 state = new AtomicReference(EmbeddedDatabaseManagerStatus.UNINITIALIZED); + private final ReadWriteLock databaseStructureLock = new ReentrantReadWriteLock(); + private final EmbeddedDatabaseManagerContext context; + private final AtomicReference engine = new AtomicReference<>(); + private final List> 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 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 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(); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerBuilder.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerBuilder.java new file mode 100644 index 0000000000..e773dd1763 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerBuilder.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerContext.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerContext.java new file mode 100644 index 0000000000..456dd64bbe --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerContext.java @@ -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 getTableDefinitions(); +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerStatus.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerStatus.java new file mode 100644 index 0000000000..d0cdff6c7c --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerStatus.java @@ -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 +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockUnsuccessfulException.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockUnsuccessfulException.java new file mode 100644 index 0000000000..c0cc4a006e --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockUnsuccessfulException.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockedClient.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockedClient.java new file mode 100644 index 0000000000..636cd76ddc --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/LockedClient.java @@ -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 query(final String query, final QueryResultProcessor rowProcessor) throws DatabaseException { + return lockedOperation(() -> client.query(query, rowProcessor)); + } + + @Override + public void disconnect() throws DatabaseException { + client.disconnect(); + } + + private R lockedOperation(final Callable 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 + + '}'; + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ManagedTableDefinition.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ManagedTableDefinition.java new file mode 100644 index 0000000000..8d4e143527 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/ManagedTableDefinition.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterFlushWorker.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/NoOpClient.java similarity index 54% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterFlushWorker.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/NoOpClient.java index 4ae0c4a556..527f24f235 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/storage/BufferedWriterFlushWorker.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/NoOpClient.java @@ -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> bufferedWriterList = new ArrayList<>(); - - public BufferedWriterFlushWorker(final List> 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 query(final String query, final QueryResultProcessor rowProcessor) { + LOGGER.debug("Querying: {}", query); + return rowProcessor.getResult(); + } + + @Override + public void disconnect() { + LOGGER.debug("Disconnecting"); } } diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RetryingClient.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RetryingClient.java new file mode 100644 index 0000000000..6f2631372b --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RetryingClient.java @@ -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) 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) context -> { + fallbackClient.insert(tableName, rowSource); + return null; + } + ); + } + + @Override + public T query(final String query, final QueryResultProcessor 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 implements RetryCallback { + @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 errorAction, final Client client, final Client fallbackClient) { + final RetryListener listener = new RetryListener() { + @Override + public void onError(final RetryContext context, final RetryCallback 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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RolloverWorker.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RolloverWorker.java new file mode 100644 index 0000000000..504f19768d --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/RolloverWorker.java @@ -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 tableDefinitions = new HashSet<>(); + + RolloverWorker(final Client client, final Set 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"); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/SimpleEmbeddedDatabaseManagerContext.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/SimpleEmbeddedDatabaseManagerContext.java new file mode 100644 index 0000000000..6975be5ccd --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/SimpleEmbeddedDatabaseManagerContext.java @@ -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 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 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 + + '}'; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/SqlExecutionContextFactory.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/SqlExecutionContextFactory.java similarity index 89% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/SqlExecutionContextFactory.java rename to nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/SqlExecutionContextFactory.java index 5ac6c19d1a..1b206fbb6d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/nifi-framework-questdb-status-history/src/main/java/org/apache/nifi/controller/status/history/questdb/SqlExecutionContextFactory.java +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/SqlExecutionContextFactory.java @@ -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); } } diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/TableWriterBasedInsertRowContext.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/TableWriterBasedInsertRowContext.java new file mode 100644 index 0000000000..e1ed645e52 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/embedded/TableWriterBasedInsertRowContext.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMapping.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMapping.java new file mode 100644 index 0000000000..8c8dd3a5cb --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMapping.java @@ -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 getNewInstance(); + int getNumberOfFields(); + Class getFieldType(int position); + BiConsumer getMapping(int position); + + static QueryResultProcessor> getResultProcessor(final RequestMapping mapping) { + return new RequestMappingBasedQueryResultProcessor<>(mapping); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBasedQueryResultProcessor.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBasedQueryResultProcessor.java new file mode 100644 index 0000000000..97d6232373 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBasedQueryResultProcessor.java @@ -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 implements QueryResultProcessor> { + private final RequestMapping mapping; + private final List result = new ArrayList<>(); + + RequestMappingBasedQueryResultProcessor(final RequestMapping 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 integerRecordBiFunction = ENTRY_FILLERS.get(mapping.getFieldType(position)); + mapping.getMapping(position).accept(entry, integerRecordBiFunction.apply(position, context)); + } + + result.add(entry); + } + + @Override + public List getResult() { + return result; + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBuilder.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBuilder.java new file mode 100644 index 0000000000..cfc13a3035 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/RequestMappingBuilder.java @@ -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 { + private final Supplier factory; + private final List, BiConsumer>> fieldMappings = new LinkedList<>(); + + private RequestMappingBuilder(final Supplier factory) { + this.factory = factory; + } + + public RequestMappingBuilder addField(final Class type, final BiConsumer mapping) { + fieldMappings.add(Pair.of(type, (BiConsumer) mapping)); + return this; + } + + public RequestMappingBuilder addStringField(final BiConsumer mapping) { + return addField(String.class, mapping); + } + + public RequestMappingBuilder addInstantField(final BiConsumer mapping) { + return addField(Instant.class, mapping); + } + + public RequestMappingBuilder addLongField(final BiConsumer mapping) { + return addField(Long.class, mapping); + } + + public RequestMappingBuilder addIntegerField(final BiConsumer mapping) { + return addField(Integer.class, mapping); + } + + public static RequestMappingBuilder of(final Supplier factory) { + return new RequestMappingBuilder<>(factory); + } + + public RequestMapping build() { + if (fieldMappings.isEmpty()) { + throw new IllegalArgumentException("There must be at least one declared field"); + } + + return new SimpleRequestMapping(factory, fieldMappings); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/SimpleRequestMapping.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/SimpleRequestMapping.java new file mode 100644 index 0000000000..cc93dbda3f --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/mapping/SimpleRequestMapping.java @@ -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 implements RequestMapping { + private final Supplier factory; + private final List, BiConsumer>> fieldMappings; + + SimpleRequestMapping( + final Supplier factory, + final List, BiConsumer>> 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 getMapping(final int position) { + return fieldMappings.get(position).getRight(); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/package-info.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/package-info.java new file mode 100644 index 0000000000..96d2bc0af0 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/package-info.java @@ -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. + */ + +/** + *

+ * 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. + *

+ * + *

+ * This has multiple purposes such as: + *

+ * + *
    + *
  • Making version changes of the underlying library easier.
  • + *
  • Preventing dependency sprawl.
  • + *
  • Hiding usage details.
  • + *
+ * + *

+ * The main entry point for the DatabaseManager which is responsible for returning a Client which + * might be used for the NiFi code to interact with the QuestDB instance. + *

+ * + *

Embedded implementation

+ * + *

+ * Currently the bundle supports connecting to an embedded QuestDB instance which is managed by the EmbeddedDatabaseManager. + * This includes creating the database and necessary tables and also rolling over old data. In order to up an embedded database a + * properly parametrized EmbeddedDatabaseManagerContext is necessary with the ManagedTableDefinition + * instances in place. + *

+ * + *

+ * Users can specify table definition and roll over strategy via ManagedTableDefinition 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. + *

+ * + * + + *

Mapping

+ * + *

+ * 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 RequestMappingBuilder. It can be wrapped by RequestMapping#getResultProcessor and used as any + * QueryResultProcessor. + *

+ * + *

Error handling

+ * + *

+ * The bundle provides error handling using various exceptions all derived from DatabaseException. 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. + *

+ */ +package org.apache.nifi.questdb; diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategy.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategy.java new file mode 100644 index 0000000000..a5717eef4b --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategy.java @@ -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 timeSource; + private final int daysToKeep; + + DeleteOldRolloverStrategy(final Supplier timeSource, final int daysToKeep) { + this.timeSource = timeSource; + this.daysToKeep = daysToKeep; + } + + @Override + public void rollOver(final Client client, final String tableName) { + try { + final List 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 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 result = new LinkedList<>(); + + @Override + public void processRow(final QueryRowContext context) { + result.add(context.getString(0)); + } + + @Override + public List getResult() { + Collections.sort(result); + return result; + } + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/KeepAllRolloverStrategy.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/KeepAllRolloverStrategy.java new file mode 100644 index 0000000000..93f52fa776 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/KeepAllRolloverStrategy.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/RolloverStrategy.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/RolloverStrategy.java new file mode 100644 index 0000000000..5da72aa910 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/main/java/org/apache/nifi/questdb/rollover/RolloverStrategy.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/CompositeClientTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/CompositeClientTest.java new file mode 100644 index 0000000000..c6a26a4f7f --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/CompositeClientTest.java @@ -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 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)); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ConditionAwareClientTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ConditionAwareClientTest.java new file mode 100644 index 0000000000..4c2b60d544 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ConditionAwareClientTest.java @@ -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> 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)); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedClientTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedClientTest.java new file mode 100644 index 0000000000..e26f1776b5 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedClientTest.java @@ -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 testEvents = QuestDbTestUtil.getTestData(); + final Client client = getTestSubject(); + client.execute(CREATE_EVENT_TABLE); + + client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testEvents)); + final Iterable result = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + + assertQueryResultMatchesWithInserts(result, testEvents); + } + + @Test + public void testInsertAndQueryWhenFieldOrderIsDifferent() throws DatabaseException { + final List testEvents = QuestDbTestUtil.getTestData(); + final Client client = getTestSubject(); + client.execute(CREATE_EVENT2_TABLE); + + client.insert(EVENT2_TABLE_NAME, QuestDbTestUtil.getEventTableDataSourceWithDifferentOrder(testEvents)); + final Iterable 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 result, final List testEvents) { + assertNumberOfEntities(3, result); + final Iterator 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 iterable) { + final AtomicInteger counted = new AtomicInteger(0); + iterable.forEach(e -> counted.incrementAndGet()); + assertEquals(expectedNumber, counted.get()); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerTest.java new file mode 100644 index 0000000000..0ca65d2c5c --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedDatabaseManagerTest.java @@ -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 testData = getTestData(); + final DatabaseManager testSubject = getTestSubject(); + assertDatabaseFolderIsNotEmpty(); + + final Client client = testSubject.acquireClient(); + + client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData)); + final List 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 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 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 testData = getTestData(); + final DatabaseManager testSubject = getTestSubject(); + + final Client client1 = testSubject.acquireClient(); + client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData)); + final List result1 = client1.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + + final Client client2 = testSubject.acquireClient(); + final List 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 testData = getTestData(); + final DatabaseManager testSubject = getTestSubject(); + final CountDownLatch step1 = new CountDownLatch(1); + final CountDownLatch step2 = new CountDownLatch(1); + final AtomicReference> result1 = new AtomicReference<>(); + final AtomicReference> 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 testData = getTestData(); + final DatabaseManager testSubject1 = getTestSubject(); + + final Client client1 = testSubject1.acquireClient(); + client1.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(testData)); + final List 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTest.java new file mode 100644 index 0000000000..a1e86e0cec --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTest.java @@ -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()); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTestUtil.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTestUtil.java new file mode 100644 index 0000000000..793a339200 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/EmbeddedQuestDbTestUtil.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/LockedClientTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/LockedClientTest.java new file mode 100644 index 0000000000..ac2a7cf16b --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/LockedClientTest.java @@ -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 query(final String query, final QueryResultProcessor 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 query(final String query, final QueryResultProcessor rowProcessor) throws DatabaseException { + try { + return client.query(query, rowProcessor); + } finally { + step3countDownLatch.countDown(); + } + } + + @Override + public void disconnect() {} + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ManagedQuestDbTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ManagedQuestDbTest.java new file mode 100644 index 0000000000..6e23786dae --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/ManagedQuestDbTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.questdb.embedded; + +import io.questdb.cairo.CairoEngine; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +public abstract class ManagedQuestDbTest extends EmbeddedQuestDbTest { + protected CairoEngine engine; + + @BeforeEach + public void setUp() { + engine = EmbeddedQuestDbTestUtil.getEngine(testDbPathDirectory.toAbsolutePath().toString()); + } + + @AfterEach + public void tearDown() { + engine.close(); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/RetryingClientTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/RetryingClientTest.java new file mode 100644 index 0000000000..f05809df74 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/embedded/RetryingClientTest.java @@ -0,0 +1,194 @@ +/* + * 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.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +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 RetryingClientTest { + + @Mock + Client fallbackClient; + + private TestClient testQuestDbClient; + + private Client testSubject; + + @Test + public void testCompileRealClientIsCalledOnceWhenNoError() throws DatabaseException { + setUpTestSubject(0); + + testSubject.execute(QuestDbTestUtil.SELECT_QUERY); + + assertEquals(1, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, never()).execute(anyString()); + } + + @Test + public void testCompileRealClientIsCalledTwiceWhenThereIsError() throws DatabaseException { + setUpTestSubject(1); + + testSubject.execute(QuestDbTestUtil.SELECT_QUERY); + + assertEquals(2, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, never()).execute(anyString()); + } + + @Test + public void testCompileFallbackIsCalledWhenErrorRemains() throws DatabaseException { + setUpTestSubject(3); + + testSubject.execute(QuestDbTestUtil.SELECT_QUERY); + + assertEquals(3, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, times(1)).execute(QuestDbTestUtil.SELECT_QUERY); + } + + @Test + public void testQueryRealClientIsCalledOnceWhenNoError() throws DatabaseException { + setUpTestSubject(0); + + testSubject.query(QuestDbTestUtil.SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + + assertEquals(1, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, never()).query(anyString(), any(QueryResultProcessor.class)); + } + + @Test + public void testQueryRealClientIsCalledTwiceWhenThereIsError() throws DatabaseException { + setUpTestSubject(1); + + testSubject.query(QuestDbTestUtil.SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + + assertEquals(2, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, never()).query(anyString(), any(QueryResultProcessor.class)); + } + + @Test + public void testQueryFallbackIsCalledWhenErrorRemains() throws DatabaseException { + setUpTestSubject(3); + + final QueryResultProcessor> queryResultProcessor = RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING); + testSubject.query(QuestDbTestUtil.SELECT_QUERY, queryResultProcessor); + + assertEquals(3, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, times(1)).query(QuestDbTestUtil.SELECT_QUERY, queryResultProcessor); + } + + @Test + public void testInsertRealClientIsCalledOnceWhenNoError() throws DatabaseException { + setUpTestSubject(0); + final List events = QuestDbTestUtil.getTestData(); + + testSubject.insert(QuestDbTestUtil.EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(events)); + + assertEquals(1, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, never()).insert(anyString(), any(InsertRowDataSource.class)); + } + + @Test + public void testInsertRealClientIsCalledTwiceWhenThereIsError() throws DatabaseException { + setUpTestSubject(1); + final List events = QuestDbTestUtil.getTestData(); + + testSubject.insert(QuestDbTestUtil.EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(events)); + + assertEquals(2, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, never()).insert(anyString(), any(InsertRowDataSource.class)); + } + + @Test + public void testInsertFallbackIsCalledWhenErrorRemains() throws DatabaseException { + setUpTestSubject(3); + final List events = QuestDbTestUtil.getTestData(); + + final InsertRowDataSource insertRowDataSource = QuestDbTestUtil.getEventTableDataSource(events); + testSubject.insert(QuestDbTestUtil.EVENT_TABLE_NAME, insertRowDataSource); + + assertEquals(3, testQuestDbClient.getNumberOfCalls()); + verify(fallbackClient, times(1)).insert(QuestDbTestUtil.EVENT_TABLE_NAME, insertRowDataSource); + } + + private void setUpTestSubject(final int numberOfErrors) { + testQuestDbClient = new TestClient(numberOfErrors); + testSubject = RetryingClient.getInstance(2, (i, e) -> {}, testQuestDbClient, fallbackClient); + } + + private static class TestClient implements Client { + private final int numberOfErrors; + private int numberOfCalls = 0; + + private TestClient(final int numberOfErrors) { + this.numberOfErrors = numberOfErrors; + } + + @Override + public void execute(final String query) throws DatabaseException { + numberOfCalls++; + + if (numberOfCalls <= numberOfErrors) { + throw new DatabaseException("Test exception"); + } + } + + @Override + public void insert(final String tableName, final InsertRowDataSource rowSource) throws DatabaseException { + numberOfCalls++; + + if (numberOfCalls <= numberOfErrors) { + throw new DatabaseException("Test exception"); + } + } + + @Override + public T query(final String query, final QueryResultProcessor rowProcessor) throws DatabaseException { + numberOfCalls++; + + if (numberOfCalls <= numberOfErrors) { + throw new DatabaseException("Test exception"); + } + + return rowProcessor.getResult(); + } + + @Override + public void disconnect() {} + + public int getNumberOfCalls() { + return numberOfCalls; + } + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/mapping/RequestMappingBuilderTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/mapping/RequestMappingBuilderTest.java new file mode 100644 index 0000000000..bb69d647b9 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/mapping/RequestMappingBuilderTest.java @@ -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.mapping; + +import org.apache.nifi.questdb.util.Event; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class RequestMappingBuilderTest { + + @Test + public void testCreateQueryMappingWithoutFields() { + assertThrows(IllegalArgumentException.class, () -> RequestMappingBuilder.of(Event::new).build()); + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategyTest.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategyTest.java new file mode 100644 index 0000000000..14a9d76793 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/rollover/DeleteOldRolloverStrategyTest.java @@ -0,0 +1,90 @@ +/* + * 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.DatabaseException; +import org.apache.nifi.questdb.embedded.EmbeddedQuestDbTestUtil; +import org.apache.nifi.questdb.embedded.ManagedQuestDbTest; +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.time.Instant; +import java.time.ZonedDateTime; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.StreamSupport; + +import static org.apache.nifi.questdb.util.QuestDbTestUtil.CREATE_EVENT_TABLE; +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; + +class DeleteOldRolloverStrategyTest extends ManagedQuestDbTest { + + @Test + public void testCleaningUpOldData() throws DatabaseException { + final Client client = getTestClient(); + final DeleteOldRolloverStrategy testSubject = new DeleteOldRolloverStrategy(ZonedDateTime::now, 1); + + final List entries = new ArrayList<>(); + entries.add(new Event(Instant.now().minus(2, ChronoUnit.DAYS), "A", 1)); + entries.add(new Event(Instant.now(), "B", 2)); + + client.execute(CREATE_EVENT_TABLE); + client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(entries)); + + final Iterable resultBeforeRollover = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + assertEquals(2, StreamSupport.stream(resultBeforeRollover.spliterator(), false).count()); + + testSubject.rollOver(client, EVENT_TABLE_NAME); + + final Iterable resultAfterRollover = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + assertEquals(1, StreamSupport.stream(resultAfterRollover.spliterator(), false).count()); + + // Multiple runs should not delete further data + testSubject.rollOver(client, EVENT_TABLE_NAME); + testSubject.rollOver(client, EVENT_TABLE_NAME); + + final Iterable resultAfterMultipleRollovers = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + assertEquals(1, StreamSupport.stream(resultAfterMultipleRollovers.spliterator(), false).count()); + } + + @Test + public void testKeepingOldDataIfItIsTheLastPartition() throws DatabaseException { + final Client client = getTestClient(); + final DeleteOldRolloverStrategy testSubject = new DeleteOldRolloverStrategy(ZonedDateTime::now, 1); + + final List entries = new ArrayList<>(); + entries.add(new Event(Instant.now().minus(2, ChronoUnit.DAYS), "A", 1)); + + client.execute(CREATE_EVENT_TABLE); + client.insert(EVENT_TABLE_NAME, QuestDbTestUtil.getEventTableDataSource(entries)); + + testSubject.rollOver(client, EVENT_TABLE_NAME); + + final Iterable resultAfterRollover = client.query(SELECT_QUERY, RequestMapping.getResultProcessor(QuestDbTestUtil.EVENT_TABLE_REQUEST_MAPPING)); + assertEquals(1, StreamSupport.stream(resultAfterRollover.spliterator(), false).count()); + } + + private Client getTestClient() { + return EmbeddedQuestDbTestUtil.getEmbeddedClient(engine); + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/Event.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/Event.java new file mode 100644 index 0000000000..ba0fe53ab0 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/Event.java @@ -0,0 +1,85 @@ +/* + * 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.util; + +import java.time.Instant; +import java.util.Objects; + +/** + * This class exists for test purposes + */ +public class Event { + private Instant captured; + private String subject; + private long value; + + public Event() {} + + public Event(final Instant captured, final String subject, final long value) { + this.captured = captured; + this.subject = subject; + this.value = value; + } + + public String getSubject() { + return subject; + } + + public void setSubject(final String subject) { + this.subject = subject; + } + + public long getValue() { + return value; + } + + public void setValue(final long value) { + this.value = value; + } + + public Instant getCaptured() { + return captured; + } + + public void setCaptured(final Instant captured) { + this.captured = captured; + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + final Event event = (Event) o; + return value == event.value && Objects.equals(captured, event.captured) && Objects.equals(subject, event.subject); + } + + @Override + public int hashCode() { + return Objects.hash(captured, subject, value); + } + + @Override + public String toString() { + return "org.apache.nifi.questdb.util.Event{" + + "captured=" + captured + + ", subject='" + subject + '\'' + + ", value=" + value + + '}'; + } + + +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/QuestDbTestUtil.java b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/QuestDbTestUtil.java new file mode 100644 index 0000000000..d480657971 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/java/org/apache/nifi/questdb/util/QuestDbTestUtil.java @@ -0,0 +1,118 @@ +/* + * 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.util; + +import org.apache.nifi.questdb.InsertRowContext; +import org.apache.nifi.questdb.InsertRowDataSource; +import org.apache.nifi.questdb.mapping.RequestMapping; +import org.apache.nifi.questdb.mapping.RequestMappingBuilder; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; + +public class QuestDbTestUtil { + public static final String EVENT_TABLE_NAME = "event"; + public static final String EVENT2_TABLE_NAME = "event2"; + public static final String CREATE_EVENT_TABLE = + "CREATE TABLE event (" + + "captured TIMESTAMP," + + "subject SYMBOL capacity 256 nocache," + + "value LONG" + + ") TIMESTAMP(captured) PARTITION BY DAY"; + // Deliberately similar to the Event table but different column order + public static final String CREATE_EVENT2_TABLE = + "CREATE TABLE event2 (" + + "subject SYMBOL capacity 256 nocache," + + "captured TIMESTAMP," + + "value LONG" + + ") TIMESTAMP(captured) PARTITION BY DAY"; + public static final String SELECT_QUERY = "SELECT * FROM event"; + public static final String SELECT_QUERY_2 = "SELECT * FROM event2"; + + public static InsertRowDataSource getEventTableDataSource(final Collection events) { + return new InsertRowDataSource() { + private final Iterator eventsToInsert = events.iterator(); + + @Override + public boolean hasNextToInsert() { + return eventsToInsert.hasNext(); + } + + @Override + public void fillRowData(final InsertRowContext context) { + final Event event = eventsToInsert.next(); + context.initializeRow(event.getCaptured()) + .addString(1, event.getSubject()) + .addLong(2, event.getValue()); + } + }; + } + + public static InsertRowDataSource getEventTableDataSourceWithDifferentOrder(final Collection events) { + return new InsertRowDataSource() { + private final Iterator eventsToInsert = events.iterator(); + + @Override + public boolean hasNextToInsert() { + return eventsToInsert.hasNext(); + } + + @Override + public void fillRowData(final InsertRowContext context) { + final Event event = eventsToInsert.next(); + context.initializeRow(event.getCaptured()) + .addString(0, event.getSubject()) + .addLong(2, event.getValue()); + } + }; + } + + public static final RequestMapping EVENT_TABLE_REQUEST_MAPPING = RequestMappingBuilder.of(Event::new) + .addInstantField(Event::setCaptured) + .addStringField(Event::setSubject) + .addLongField(Event::setValue) + .build(); + + public static final RequestMapping EVENT_TABLE_REQUEST_MAPPING_DIFFERENT_ORDER = RequestMappingBuilder.of(Event::new) + .addStringField(Event::setSubject) + .addInstantField(Event::setCaptured) + .addLongField(Event::setValue) + .build(); + + public static List getTestData() { + // Using Instant.new() will result some difference on nanosecond level + final Event event1 = new Event(Instant.ofEpochMilli(System.currentTimeMillis()), "subject1", 1); + final Event event2 = new Event(Instant.ofEpochMilli(System.currentTimeMillis()), "subject2", 2); + final Event event3 = new Event(Instant.ofEpochMilli(System.currentTimeMillis()), "subject1", 3); + return Arrays.asList(event1, event2, event3); + } + + public static List getRandomTestData() { + final List result = new ArrayList<>(100); + + for (int i = 0; i <= 100; i++) { + result.add(new Event(Instant.ofEpochMilli(System.currentTimeMillis()), UUID.randomUUID().toString(), i)); + } + + return result; + } +} diff --git a/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/resources/log-stdout.conf b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/resources/log-stdout.conf new file mode 100644 index 0000000000..4fb020ec29 --- /dev/null +++ b/nifi-nar-bundles/nifi-questdb-bundle/nifi-questdb/src/test/resources/log-stdout.conf @@ -0,0 +1,18 @@ +# 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. + +writers=stdout +w.stdout.class=io.questdb.log.LogConsoleWriter +w.stdout.level=CRITICAL diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/pom.xml b/nifi-nar-bundles/nifi-questdb-bundle/pom.xml similarity index 83% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/pom.xml rename to nifi-nar-bundles/nifi-questdb-bundle/pom.xml index 0be787b153..2ebd1a0094 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-questdb-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-questdb-bundle/pom.xml @@ -17,13 +17,14 @@ 4.0.0 org.apache.nifi - nifi-framework + nifi-nar-bundles 2.0.0-SNAPSHOT - nifi-framework-questdb-bundle + nifi-questdb-bundle pom - nifi-framework-questdb-status-history - nifi-framework-questdb-nar + nifi-questdb-status-history + nifi-questdb-status-history-nar + nifi-questdb diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml index bb5fb7dac7..2ffa8a1392 100755 --- a/nifi-nar-bundles/pom.xml +++ b/nifi-nar-bundles/pom.xml @@ -115,8 +115,23 @@ nifi-opentelemetry-bundle nifi-apicurio-bundle nifi-jolt-bundle + nifi-questdb-bundle + + + + shibboleth + https://build.shibboleth.net/nexus/content/repositories/releases/ + + true + + + false + + + +