From 22ad7d542d627e767f962b13236da90a0d6410f5 Mon Sep 17 00:00:00 2001 From: exceptionfactory Date: Thu, 12 Oct 2023 13:50:25 -0500 Subject: [PATCH] NIFI-12206 Refactor Flow History using JetBrains Xodus (#7870) * NIFI-12206 Refactored Flow History using JetBrains Xodus - Replaced H2 Database Engine with JetBrains Xodus for persistent storage of FlowConfigurationHistory - Added EntityStoreAuditService implementation using Xodus PersistentEntityStore - Removed nifi.h2.url.append from properties --- .../service/MiNiFiPropertiesGenerator.java | 1 - .../MINIFI-216/nifi.properties.before | 3 +- .../MINIFI-245/nifi.properties.before | 3 +- .../test/resources/MINIFI-277/nifi.properties | 3 +- .../NIFI-8753/nifi.properties.before | 3 +- .../src/test/resources/conf/nifi.properties | 3 +- .../org/apache/nifi/util/NiFiProperties.java | 1 - .../NiFiProperties/conf/nifi.blank.properties | 3 +- .../conf/nifi.missing.properties | 1 - .../NiFiProperties/conf/nifi.properties | 3 +- .../main/asciidoc/administration-guide.adoc | 5 +- .../nifi-administration/pom.xml | 49 +- .../admin/AuditDataSourceFactoryBean.java | 228 ---- .../apache/nifi/admin/RepositoryUtils.java | 90 -- .../org/apache/nifi/admin/dao/ActionDAO.java | 83 -- .../org/apache/nifi/admin/dao/DAOFactory.java | 25 - .../admin/dao/impl/StandardActionDAO.java | 1006 ----------------- .../service/EntityStoreAuditService.java | 636 +++++++++++ .../service/action/AddActionsAction.java | 48 - .../service/action/DeletePreviousValues.java | 44 - .../service/action/GetActionsAction.java | 48 - .../service/action/GetPreviousValues.java | 43 - .../service/action/PurgeActionsAction.java | 51 - .../ActionEntity.java} | 41 +- .../ActionLink.java} | 29 +- .../entity/ConfigureDetailsEntity.java} | 38 +- .../service/entity/ConnectDetailsEntity.java | 49 + .../EntityProperty.java} | 14 +- .../EntityType.java} | 38 +- .../entity/ExtensionDetailsEntity.java} | 22 +- .../service/entity/MoveDetailsEntity.java | 43 + .../entity/PurgeDetailsEntity.java} | 24 +- .../service/impl/StandardAuditService.java | 255 ----- .../service/transaction/Transaction.java | 49 - .../transaction/impl/StandardTransaction.java | 91 -- .../impl/StandardTransactionBuilder.java | 59 - .../resources/nifi-administration-context.xml | 36 - .../service/EntityStoreAuditServiceTest.java | 369 ++++++ .../src/test/resources/conf/nifi.properties | 3 +- .../src/test/resources/conf/nifi.properties | 3 +- .../nifi-framework-core/pom.xml | 12 +- .../src/test/resources/conf/nifi.properties | 3 +- .../flowcontrollertest.nifi.properties | 3 +- .../resources/nifi-with-remote.properties | 3 +- ...standardflowserializertest.nifi.properties | 3 +- ...andardflowsynchronizerspec.nifi.properties | 3 +- ...andardprocessschedulertest.nifi.properties | 3 +- .../conf/nifi.nar_with_native_lib.properties | 3 +- .../nifi.nar_without_native_lib.properties | 3 +- .../src/test/resources/conf/nifi.properties | 3 +- .../NarUnpacker/conf/nifi.properties | 3 +- .../src/test/resources/nifi.properties | 3 +- .../nifi/headless/HeadlessAuditService.java | 68 ++ .../nifi/headless/HeadlessNiFiServer.java | 3 +- .../src/test/resources/nifi.properties | 3 +- .../nifi-framework/nifi-resources/pom.xml | 2 - .../src/main/resources/conf/logback.xml | 3 + .../src/main/resources/conf/nifi.properties | 3 +- .../conf/encrypted.nifi.properties | 3 +- .../NiFiProperties/conf/nifi.properties | 3 +- .../src/test/resources/nifi.properties | 4 - .../nifi/web/NiFiWebApiConfiguration.java | 20 +- .../nifi-anonymous-allowed.properties | 3 +- .../access-control/nifi-flow.properties | 3 +- .../nifi-mapped-identities.properties | 3 +- .../resources/access-control/nifi.properties | 3 +- .../resources/site-to-site/nifi.properties | 3 +- .../nifi-framework-bundle/pom.xml | 5 - .../conf/clustered/node1/nifi.properties | 3 +- .../conf/clustered/node2/nifi.properties | 3 +- .../resources/conf/default/nifi.properties | 3 +- .../resources/conf/pythonic/nifi.properties | 3 +- .../test/resources/nifi_default.properties | 3 +- ...sitive_properties_protected_aes.properties | 3 +- ...ve_properties_protected_aes_128.properties | 3 +- ...operties_protected_aes_password.properties | 3 +- ...ties_protected_aes_password_128.properties | 3 +- ...ensitive_properties_unprotected.properties | 3 +- ...ed_and_empty_protection_schemes.properties | 3 +- .../test/resources/localhost/nifi.properties | 3 +- 80 files changed, 1358 insertions(+), 2398 deletions(-) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/EntityStoreAuditService.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DeletePreviousValues.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/{transaction/TransactionException.java => entity/ActionEntity.java} (57%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/{action/GetActionAction.java => entity/ActionLink.java} (61%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/{user/NiFiUserGroup.java => admin/service/entity/ConfigureDetailsEntity.java} (63%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ConnectDetailsEntity.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/{transaction/TransactionBuilder.java => entity/EntityProperty.java} (75%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/{action/AdministrationAction.java => entity/EntityType.java} (60%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/{dao/DataAccessException.java => service/entity/ExtensionDetailsEntity.java} (66%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/MoveDetailsEntity.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/{dao/impl/DAOFactoryImpl.java => service/entity/PurgeDetailsEntity.java} (66%) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/EntityStoreAuditServiceTest.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessAuditService.java diff --git a/minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiPropertiesGenerator.java b/minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiPropertiesGenerator.java index 16a393d0c9..939ded36f7 100644 --- a/minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiPropertiesGenerator.java +++ b/minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiPropertiesGenerator.java @@ -99,7 +99,6 @@ public class MiNiFiPropertiesGenerator { Triple.of(NiFiProperties.STATE_MANAGEMENT_CONFIG_FILE, "./conf/state-management.xml", "# State Management"), Triple.of(NiFiProperties.STATE_MANAGEMENT_LOCAL_PROVIDER_ID, "local-provider", "# The ID of the local state provider"), Triple.of(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY, "./database_repository", "# H2 Settings"), - Triple.of(NiFiProperties.H2_URL_APPEND, ";LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE", EMPTY), Triple.of(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository", "# FlowFile Repository"), Triple.of(NiFiProperties.FLOWFILE_REPOSITORY_DIRECTORY, "./flowfile_repository", EMPTY), Triple.of(NiFiProperties.FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL, "20 secs", EMPTY), diff --git a/minifi/minifi-bootstrap/src/test/resources/MINIFI-216/nifi.properties.before b/minifi/minifi-bootstrap/src/test/resources/MINIFI-216/nifi.properties.before index 8df7e966ca..aa84785cae 100644 --- a/minifi/minifi-bootstrap/src/test/resources/MINIFI-216/nifi.properties.before +++ b/minifi/minifi-bootstrap/src/test/resources/MINIFI-216/nifi.properties.before @@ -41,9 +41,8 @@ nifi.state.management.configuration.file=./conf/state-management.xml # The ID of the local state provider nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/minifi/minifi-bootstrap/src/test/resources/MINIFI-245/nifi.properties.before b/minifi/minifi-bootstrap/src/test/resources/MINIFI-245/nifi.properties.before index 8df7e966ca..aa84785cae 100644 --- a/minifi/minifi-bootstrap/src/test/resources/MINIFI-245/nifi.properties.before +++ b/minifi/minifi-bootstrap/src/test/resources/MINIFI-245/nifi.properties.before @@ -41,9 +41,8 @@ nifi.state.management.configuration.file=./conf/state-management.xml # The ID of the local state provider nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/minifi/minifi-bootstrap/src/test/resources/MINIFI-277/nifi.properties b/minifi/minifi-bootstrap/src/test/resources/MINIFI-277/nifi.properties index 8df7e966ca..aa84785cae 100644 --- a/minifi/minifi-bootstrap/src/test/resources/MINIFI-277/nifi.properties +++ b/minifi/minifi-bootstrap/src/test/resources/MINIFI-277/nifi.properties @@ -41,9 +41,8 @@ nifi.state.management.configuration.file=./conf/state-management.xml # The ID of the local state provider nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/minifi/minifi-bootstrap/src/test/resources/NIFI-8753/nifi.properties.before b/minifi/minifi-bootstrap/src/test/resources/NIFI-8753/nifi.properties.before index 8df7e966ca..aa84785cae 100644 --- a/minifi/minifi-bootstrap/src/test/resources/NIFI-8753/nifi.properties.before +++ b/minifi/minifi-bootstrap/src/test/resources/NIFI-8753/nifi.properties.before @@ -41,9 +41,8 @@ nifi.state.management.configuration.file=./conf/state-management.xml # The ID of the local state provider nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/minifi/minifi-integration-tests/src/test/resources/conf/nifi.properties b/minifi/minifi-integration-tests/src/test/resources/conf/nifi.properties index 85f8471bd0..66ad9e80bd 100644 --- a/minifi/minifi-integration-tests/src/test/resources/conf/nifi.properties +++ b/minifi/minifi-integration-tests/src/test/resources/conf/nifi.properties @@ -30,9 +30,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo 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 78432ce444..dd8ddcf700 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 @@ -74,7 +74,6 @@ public class NiFiProperties extends ApplicationProperties { public static final String COMPONENT_DOCS_DIRECTORY = "nifi.documentation.working.directory"; public static final String SENSITIVE_PROPS_KEY = "nifi.sensitive.props.key"; public static final String SENSITIVE_PROPS_ALGORITHM = "nifi.sensitive.props.algorithm"; - public static final String H2_URL_APPEND = "nifi.h2.url.append"; public static final String REMOTE_INPUT_HOST = "nifi.remote.input.host"; public static final String REMOTE_INPUT_PORT = "nifi.remote.input.socket.port"; public static final String SITE_TO_SITE_SECURE = "nifi.remote.input.secure"; diff --git a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.blank.properties b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.blank.properties index 64dd766aa8..83536452b7 100644 --- a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.blank.properties +++ b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.blank.properties @@ -29,9 +29,8 @@ nifi.nar.library.directory= nifi.custom.nar.library.directory.alt= nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.missing.properties b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.missing.properties index 31f0ef53a3..638613bc78 100644 --- a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.missing.properties +++ b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.missing.properties @@ -29,7 +29,6 @@ nifi.nar.working.directory=./target/work/nar/ # H2 Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties index 853395b79e..833b515432 100644 --- a/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties +++ b/nifi-commons/nifi-properties/src/test/resources/NiFiProperties/conf/nifi.properties @@ -29,9 +29,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc index 2cc8d07726..81bfa21bca 100644 --- a/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -3370,14 +3370,13 @@ for components to persist state. See the <> section for more i |==== -=== H2 Settings +=== Database Settings -The H2 Settings section defines the settings for the H2 database, which keeps track of user access and flow controller history. +The Database Settings section defines the settings for the internal database, which tracks flow configuration history. |==== |*Property*|*Description* |`nifi.database.directory`*|The location of the H2 database directory. The default value is `./database_repository`. -|`nifi.h2.url.append`|This property specifies additional arguments to add to the connection string for the H2 database. The default value should be used and should not be changed. It is: `;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE`. |==== [[repository-encryption-properties]] diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml index 2cbcd425c1..e71ca72af7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml @@ -21,6 +21,9 @@ 2.0.0-SNAPSHOT nifi-administration + + 2.0.1 + org.apache.nifi @@ -35,48 +38,14 @@ nifi-framework-api - org.apache.nifi - nifi-framework-core-api + org.jetbrains.xodus + xodus-openAPI + ${xodus.version} - org.apache.nifi - nifi-nar-utils - - - org.apache.nifi - nifi-utils - - - org.apache.nifi - nifi-properties - - - com.h2database - h2 - - - org.springframework - spring-beans - - - org.springframework - spring-context - - - org.springframework - spring-core - - - org.springframework - spring-aop - - - org.aspectj - aspectjweaver - - - org.apache.commons - commons-collections4 + org.jetbrains.xodus + xodus-entity-store + ${xodus.version} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java deleted file mode 100644 index 040a207dcb..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java +++ /dev/null @@ -1,228 +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.admin; - -import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.util.NiFiProperties; -import org.h2.jdbcx.JdbcConnectionPool; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.FactoryBean; - -import java.io.File; -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.SQLFeatureNotSupportedException; -import java.sql.Statement; - -/** - * - */ -public class AuditDataSourceFactoryBean implements FactoryBean { - - private static final Logger logger = LoggerFactory.getLogger(AuditDataSourceFactoryBean.class); - private static final String NF_USERNAME_PASSWORD = "nf"; - private static final int MAX_CONNECTIONS = 5; - - // database file name - private static final String AUDIT_DATABASE_FILE_NAME = "nifi-flow-audit"; - - // ------------ - // action table - // ------------ - private static final String CREATE_ACTION_TABLE = "CREATE TABLE ACTION (" - + "ID INT NOT NULL PRIMARY KEY AUTO_INCREMENT, " - + "IDENTITY VARCHAR2(4096) NOT NULL, " - + "SOURCE_ID VARCHAR2(100) NOT NULL, " - + "SOURCE_NAME VARCHAR2(1000) NOT NULL, " - + "SOURCE_TYPE VARCHAR2(1000) NOT NULL, " - + "OPERATION VARCHAR2(50) NOT NULL, " - + "ACTION_TIMESTAMP TIMESTAMP NOT NULL " - + ")"; - - // ----------------- - // component details - // ----------------- - private static final String CREATE_PROCESSOR_DETAILS_TABLE = "CREATE TABLE PROCESSOR_DETAILS (" - + "ACTION_ID INT NOT NULL PRIMARY KEY, " - + "TYPE VARCHAR2(1000) NOT NULL, " - + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" - + ")"; - - private static final String CREATE_REMOTE_PROCESS_GROUP_DETAILS_TABLE = "CREATE TABLE REMOTE_PROCESS_GROUP_DETAILS (" - + "ACTION_ID INT NOT NULL PRIMARY KEY, " - + "URI VARCHAR2(2500) NOT NULL, " - + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" - + ")"; - - // -------------- - // action details - // -------------- - private static final String CREATE_MOVE_DETAILS_TABLE = "CREATE TABLE MOVE_DETAILS (" - + "ACTION_ID INT NOT NULL PRIMARY KEY, " - + "GROUP_ID VARCHAR2(100) NOT NULL, " - + "GROUP_NAME VARCHAR2(1000) NOT NULL, " - + "PREVIOUS_GROUP_ID VARCHAR2(100) NOT NULL, " - + "PREVIOUS_GROUP_NAME VARCHAR2(1000) NOT NULL, " - + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" - + ")"; - - private static final String CREATE_CONFIGURE_DETAILS_TABLE = "CREATE TABLE CONFIGURE_DETAILS (" - + "ACTION_ID INT NOT NULL PRIMARY KEY, " - + "NAME VARCHAR2(1000) NOT NULL, " - + "\"VALUE\" VARCHAR2(5000), " - + "PREVIOUS_VALUE VARCHAR2(5000), " - + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" - + ")"; - - private static final String CREATE_CONNECT_DETAILS_TABLE = "CREATE TABLE CONNECT_DETAILS (" - + "ACTION_ID INT NOT NULL PRIMARY KEY, " - + "SOURCE_ID VARCHAR2(100) NOT NULL, " - + "SOURCE_NAME VARCHAR2(1000), " - + "SOURCE_TYPE VARCHAR2(1000) NOT NULL, " - + "RELATIONSHIP VARCHAR2(1000), " - + "DESTINATION_ID VARCHAR2(100) NOT NULL, " - + "DESTINATION_NAME VARCHAR2(1000), " - + "DESTINATION_TYPE VARCHAR2(1000) NOT NULL, " - + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" - + ")"; - - private static final String CREATE_PURGE_DETAILS_TABLE = "CREATE TABLE PURGE_DETAILS (" - + "ACTION_ID INT NOT NULL PRIMARY KEY, " - + "END_DATE TIMESTAMP NOT NULL, " - + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" - + ")"; - - private JdbcConnectionPool connectionPool; - - private NiFiProperties properties; - - @Override - public Object getObject() throws Exception { - if (connectionPool == null) { - - // locate the repository directory - String repositoryDirectoryPath = properties.getProperty(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY); - - // ensure the repository directory is specified - if (repositoryDirectoryPath == null) { - throw new NullPointerException("Database directory must be specified."); - } - - // create a handle to the repository directory - File repositoryDirectory = new File(repositoryDirectoryPath); - - // get a handle to the database file - File dbFileNoExtension = new File(repositoryDirectory, AUDIT_DATABASE_FILE_NAME); - - // format the database url - String databaseUrl = "jdbc:h2:" + dbFileNoExtension + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3"; - String databaseUrlAppend = properties.getProperty(NiFiProperties.H2_URL_APPEND); - if (StringUtils.isNotBlank(databaseUrlAppend)) { - databaseUrl += databaseUrlAppend; - } - - // create the pool - connectionPool = JdbcConnectionPool.create(databaseUrl, NF_USERNAME_PASSWORD, NF_USERNAME_PASSWORD); - connectionPool.setMaxConnections(MAX_CONNECTIONS); - - Connection connection = null; - ResultSet rs = null; - Statement statement = null; - try { - // get a connection - connection = connectionPool.getConnection(); - final boolean isAutoCommit = connection.getAutoCommit(); - if (isAutoCommit) { - try { - connection.setAutoCommit(false); - } catch (SQLFeatureNotSupportedException sfnse) { - logger.debug("setAutoCommit(false) not supported by this driver"); - } - } - - // create a statement for initializing the database - statement = connection.createStatement(); - - // determine if the tables need to be created - rs = connection.getMetaData().getTables(null, null, "ACTION", null); - if (!rs.next()) { - logger.info("Database not built for repository: " + databaseUrl + ". Building now..."); - RepositoryUtils.closeQuietly(rs); - - // action table - statement.execute(CREATE_ACTION_TABLE); - - // component details - statement.execute(CREATE_PROCESSOR_DETAILS_TABLE); - statement.execute(CREATE_REMOTE_PROCESS_GROUP_DETAILS_TABLE); - - // action details - statement.execute(CREATE_MOVE_DETAILS_TABLE); - statement.execute(CREATE_CONFIGURE_DETAILS_TABLE); - statement.execute(CREATE_CONNECT_DETAILS_TABLE); - statement.execute(CREATE_PURGE_DETAILS_TABLE); - } - - // commit any changes - connection.commit(); - } catch (SQLException sqle) { - RepositoryUtils.rollback(connection, logger); - throw sqle; - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - RepositoryUtils.closeQuietly(connection); - } - } - - return connectionPool; - } - - @Override - public Class getObjectType() { - return JdbcConnectionPool.class; - } - - @Override - public boolean isSingleton() { - return true; - } - - public void setProperties(NiFiProperties properties) { - this.properties = properties; - } - - /** - * Disposes resources. - */ - public void shutdown() { - - // shutdown the connection pool - if (connectionPool != null) { - try { - connectionPool.dispose(); - } catch (Exception e) { - logger.warn("Unable to dispose of connection pool: " + e.getMessage()); - if (logger.isDebugEnabled()) { - logger.warn(StringUtils.EMPTY, e); - } - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java deleted file mode 100644 index f678d5dbc4..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java +++ /dev/null @@ -1,90 +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.admin; - -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import org.slf4j.Logger; - -/** - * A utility class for useful methods dealing with the repository - * - */ -public class RepositoryUtils { - - public static void rollback(final Connection conn, final Logger logger) { - try { - if (null != conn) { - conn.rollback(); - } - } catch (final SQLException sqe) { - logger.warn("The following problem occurred while trying to rollback " + conn + ": " + sqe.getLocalizedMessage()); - if (logger.isDebugEnabled()) { - logger.debug("", sqe); - } - } - } - - /** - * Closes the given statement quietly - no logging, no exceptions - * - * @param statement to close - */ - public static void closeQuietly(final Statement statement) { - - if (null != statement) { - try { - statement.close(); - } catch (final SQLException se) { /*IGNORE*/ - - } - } - } - - /** - * Closes the given result set quietly - no logging, no exceptions - * - * @param resultSet to close - */ - public static void closeQuietly(final ResultSet resultSet) { - if (null != resultSet) { - try { - resultSet.close(); - } catch (final SQLException se) {/*IGNORE*/ - - } - } - } - - /** - * Closes the given connection quietly - no logging, no exceptions - * - * @param conn to close - */ - public static void closeQuietly(final Connection conn) { - if (null != conn) { - try { - conn.close(); - } catch (final SQLException se) {/*IGNORE*/ - - } - } - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java deleted file mode 100644 index 5f0d48f4a0..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java +++ /dev/null @@ -1,83 +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.admin.dao; - -import org.apache.nifi.action.Action; -import org.apache.nifi.history.History; -import org.apache.nifi.history.HistoryQuery; -import org.apache.nifi.history.PreviousValue; - -import java.util.Date; -import java.util.List; -import java.util.Map; - -/** - * Action data access. - */ -public interface ActionDAO { - - /** - * Persists the specified action. - * - * @param action to persist - * @return the created Action with it's id - * @throws DataAccessException if unable to persist - */ - Action createAction(Action action) throws DataAccessException; - - /** - * Finds all actions that meet the specified criteria. - * - * @param actionQuery query for actions - * @return History of actions - * @throws DataAccessException dae - */ - History findActions(HistoryQuery actionQuery) throws DataAccessException; - - /** - * @param componentId to get previous values of - * @return Finds the previous values for the specified property in the - * specified component. Returns empty list if there are none - */ - Map> getPreviousValues(String componentId); - - /** - * Deletes the history of a component's property. - * - * @param propertyName the name of the property - * @param componentId to delete previous values of - */ - void deletePreviousValues(String propertyName, String componentId); - - - /** - * Finds the specified action. - * - * @param actionId action identifier - * @return Action specified - * @throws DataAccessException dae - */ - Action getAction(Integer actionId) throws DataAccessException; - - /** - * Deletes all actions up to the specified end date. - * - * @param endDate date to stop deleting at - * @throws DataAccessException dae - */ - void deleteActions(Date endDate) throws DataAccessException; -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java deleted file mode 100644 index 0635179ccc..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java +++ /dev/null @@ -1,25 +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.admin.dao; - -/** - * - */ -public interface DAOFactory { - - ActionDAO getActionDAO(); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java deleted file mode 100644 index b686054916..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java +++ /dev/null @@ -1,1006 +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.admin.dao.impl; - -import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.action.Action; -import org.apache.nifi.action.Component; -import org.apache.nifi.action.FlowChangeAction; -import org.apache.nifi.action.Operation; -import org.apache.nifi.action.component.details.ComponentDetails; -import org.apache.nifi.action.component.details.ExtensionDetails; -import org.apache.nifi.action.component.details.FlowChangeExtensionDetails; -import org.apache.nifi.action.component.details.FlowChangeRemoteProcessGroupDetails; -import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; -import org.apache.nifi.action.details.ActionDetails; -import org.apache.nifi.action.details.ConfigureDetails; -import org.apache.nifi.action.details.ConnectDetails; -import org.apache.nifi.action.details.FlowChangeConfigureDetails; -import org.apache.nifi.action.details.FlowChangeConnectDetails; -import org.apache.nifi.action.details.FlowChangeMoveDetails; -import org.apache.nifi.action.details.FlowChangePurgeDetails; -import org.apache.nifi.action.details.MoveDetails; -import org.apache.nifi.action.details.PurgeDetails; -import org.apache.nifi.admin.RepositoryUtils; -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DataAccessException; -import org.apache.nifi.history.History; -import org.apache.nifi.history.HistoryQuery; -import org.apache.nifi.history.PreviousValue; - -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Date; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -/** - * - */ -public class StandardActionDAO implements ActionDAO { - - // ------------ - // action table - // ------------ - private static final String INSERT_ACTION = "INSERT INTO ACTION (" - + "IDENTITY, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, OPERATION, ACTION_TIMESTAMP" - + ") VALUES (" - + "?, " - + "?, " - + "?, " - + "?, " - + "?, " - + "? " - + ")"; - - // ----------------- - // component details - // ----------------- - private static final String INSERT_EXTENSION_DETAILS = "INSERT INTO PROCESSOR_DETAILS (" - + "ACTION_ID, TYPE" - + ") VALUES (" - + "?, " - + "?" - + ")"; - - private static final String INSERT_REMOTE_PROCESS_GROUP_DETAILS = "INSERT INTO REMOTE_PROCESS_GROUP_DETAILS (" - + "ACTION_ID, URI" - + ") VALUES (" - + "?, " - + "?" - + ")"; - - // -------------- - // action details - // -------------- - private static final String INSERT_CONFIGURE_DETAILS = "INSERT INTO CONFIGURE_DETAILS (" - + "ACTION_ID, NAME, \"VALUE\", PREVIOUS_VALUE" - + ") VALUES (" - + "?, " - + "?, " - + "?, " - + "?" - + ")"; - - private static final String INSERT_CONNECT_DETAILS = "INSERT INTO CONNECT_DETAILS (" - + "ACTION_ID, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, RELATIONSHIP, DESTINATION_ID, DESTINATION_NAME, DESTINATION_TYPE" - + ") VALUES (" - + "?, " - + "?, " - + "?, " - + "?, " - + "?, " - + "?, " - + "?, " - + "?" - + ")"; - - private static final String INSERT_MOVE_DETAILS = "INSERT INTO MOVE_DETAILS (" - + "ACTION_ID, GROUP_ID, GROUP_NAME, PREVIOUS_GROUP_ID, PREVIOUS_GROUP_NAME" - + ") VALUES (" - + "?, " - + "?, " - + "?, " - + "?, " - + "?" - + ")"; - - private static final String INSERT_PURGE_DETAILS = "INSERT INTO PURGE_DETAILS (" - + "ACTION_ID, END_DATE" - + ") VALUES (" - + "?, " - + "?" - + ")"; - - // ------------ - // action table - // ------------ - private static final String SELECT_ACTIONS = "SELECT * FROM ACTION"; - - private static final String SELECT_ACTION_COUNT = "SELECT COUNT(*) AS ACTION_COUNT FROM ACTION"; - - private static final String SELECT_ACTION_BY_ID = "SELECT * " - + "FROM ACTION " - + "WHERE " - + "ID = ?"; - - private static final String DELETE_ACTIONS = "DELETE FROM ACTION WHERE ACTION_TIMESTAMP < ?"; - - private static final String DELETE_SPECIFIC_ACTIONS = "DELETE FROM %s WHERE %s IN (SELECT ID FROM ACTION WHERE ACTION_TIMESTAMP < ?)"; - - // ----------------- - // component details - // ----------------- - private static final String SELECT_EXTENSION_DETAILS_FOR_ACTION = "SELECT * FROM PROCESSOR_DETAILS WHERE ACTION_ID = ?"; - - private static final String SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION = "SELECT * FROM REMOTE_PROCESS_GROUP_DETAILS WHERE ACTION_ID = ?"; - - // -------------- - // action details - // -------------- - private static final String SELECT_MOVE_DETAILS_FOR_ACTION = "SELECT * FROM MOVE_DETAILS WHERE ACTION_ID = ?"; - - private static final String SELECT_CONFIGURE_DETAILS_FOR_ACTION = "SELECT * FROM CONFIGURE_DETAILS WHERE ACTION_ID = ?"; - - private static final String SELECT_CONNECT_DETAILS_FOR_ACTION = "SELECT * FROM CONNECT_DETAILS WHERE ACTION_ID = ?"; - - private static final String SELECT_PURGE_DETAILS_FOR_ACTION = "SELECT * FROM PURGE_DETAILS WHERE ACTION_ID = ?"; - - // --------------- - // previous values - // --------------- - private static final String SELECT_PREVIOUSLY_CONFIGURED_FIELDS = "SELECT DISTINCT CD.NAME " - + "FROM CONFIGURE_DETAILS CD " - + "INNER JOIN ACTION A " - + "ON CD.ACTION_ID = A.ID " - + "WHERE A.SOURCE_ID = ?"; - - private static final String SELECT_PREVIOUS_VALUES = "SELECT CD.\"VALUE\", " - + "A.ACTION_TIMESTAMP, " - + "A.IDENTITY " - + "FROM CONFIGURE_DETAILS CD " - + "INNER JOIN ACTION A " - + "ON CD.ACTION_ID = A.ID " - + "WHERE A.SOURCE_ID = ? AND CD.NAME = ? " - + "ORDER BY A.ACTION_TIMESTAMP DESC " - + "LIMIT 4"; - - private static final String DELETE_PREVIOUS_VALUES = "DELETE FROM CONFIGURE_DETAILS " + - "WHERE NAME = ? " + - "AND ACTION_ID IN " + - "(SELECT ID FROM ACTION WHERE SOURCE_ID = ?)"; - private static final String ACTION_TIMESTAMP = "ACTION_TIMESTAMP"; - private static final String SOURCE_NAME = "SOURCE_NAME"; - private static final String SOURCE_TYPE = "SOURCE_TYPE"; - private static final String OPERATION = "OPERATION"; - private static final String IDENTITY = "IDENTITY"; - private static final String ACTION_ID = "ACTION_ID"; - private static final String SOURCE_ID = "SOURCE_ID"; - - private final Connection connection; - private final Map columnMap; - - public StandardActionDAO(Connection connection) { - this.connection = connection; - - // initialize the column mappings - this.columnMap = new HashMap<>(); - this.columnMap.put("timestamp", ACTION_TIMESTAMP); - this.columnMap.put("sourceName", SOURCE_NAME); - this.columnMap.put("sourceType", SOURCE_TYPE); - this.columnMap.put("operation", OPERATION); - this.columnMap.put("userIdentity", IDENTITY); - } - - @Override - public Action createAction(Action action) throws DataAccessException { - if (action.getUserIdentity() == null) { - throw new IllegalArgumentException("User cannot be null."); - } - - if (action.getTimestamp() == null) { - throw new IllegalArgumentException("Action timestamp cannot be null."); - } - - PreparedStatement statement = null; - ResultSet rs = null; - try { - // obtain a statement to insert to the action table - statement = connection.prepareStatement(INSERT_ACTION, Statement.RETURN_GENERATED_KEYS); - statement.setString(1, StringUtils.left(action.getUserIdentity(), 4096)); - statement.setString(2, action.getSourceId()); - statement.setString(3, StringUtils.left(action.getSourceName(), 1000)); - statement.setString(4, action.getSourceType().name()); - statement.setString(5, action.getOperation().name()); - statement.setTimestamp(6, new java.sql.Timestamp(action.getTimestamp().getTime())); - - // insert the action - int updateCount = statement.executeUpdate(); - - final FlowChangeAction createdAction = new FlowChangeAction(); - createdAction.setUserIdentity(action.getUserIdentity()); - createdAction.setSourceId(action.getSourceId()); - createdAction.setSourceName(action.getSourceName()); - createdAction.setSourceType(action.getSourceType()); - createdAction.setOperation(action.getOperation()); - createdAction.setTimestamp(action.getTimestamp()); - createdAction.setActionDetails(action.getActionDetails()); - createdAction.setComponentDetails(action.getComponentDetails()); - - // get the action id - rs = statement.getGeneratedKeys(); - if (updateCount == 1 && rs.next()) { - createdAction.setId(rs.getInt(1)); - } else { - throw new DataAccessException("Unable to insert action."); - } - - // close the previous statement - statement.close(); - - // determine the type of component - ComponentDetails componentDetails = createdAction.getComponentDetails(); - if (componentDetails instanceof FlowChangeExtensionDetails) { - createExtensionDetails(createdAction.getId(), (ExtensionDetails) componentDetails); - } else if (componentDetails instanceof FlowChangeRemoteProcessGroupDetails) { - createRemoteProcessGroupDetails(createdAction.getId(), (RemoteProcessGroupDetails) componentDetails); - } - - // determine the type of action - ActionDetails details = createdAction.getActionDetails(); - if (details instanceof FlowChangeConnectDetails) { - createConnectDetails(createdAction.getId(), (ConnectDetails) details); - } else if (details instanceof FlowChangeMoveDetails) { - createMoveDetails(createdAction.getId(), (MoveDetails) details); - } else if (details instanceof FlowChangeConfigureDetails) { - createConfigureDetails(createdAction.getId(), (ConfigureDetails) details); - } else if (details instanceof FlowChangePurgeDetails) { - createPurgeDetails(createdAction.getId(), (PurgeDetails) details); - } - - return createdAction; - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - } - - private void createExtensionDetails(int actionId, ExtensionDetails extensionDetails) throws DataAccessException { - PreparedStatement statement = null; - try { - // obtain a statement to insert to the extension action table - statement = connection.prepareStatement(INSERT_EXTENSION_DETAILS); - statement.setInt(1, actionId); - statement.setString(2, StringUtils.left(extensionDetails.getType(), 1000)); - - // insert the action - int updateCount = statement.executeUpdate(); - - // ensure the operation completed successfully - if (updateCount != 1) { - throw new DataAccessException("Unable to insert extension details."); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - - private void createRemoteProcessGroupDetails(int actionId, RemoteProcessGroupDetails remoteProcessGroupDetails) throws DataAccessException { - PreparedStatement statement = null; - try { - // obtain a statement to insert to the processor action table - statement = connection.prepareStatement(INSERT_REMOTE_PROCESS_GROUP_DETAILS); - statement.setInt(1, actionId); - statement.setString(2, StringUtils.left(remoteProcessGroupDetails.getUri(), 2500)); - - // insert the action - int updateCount = statement.executeUpdate(); - - // ensure the operation completed successfully - if (updateCount != 1) { - throw new DataAccessException("Unable to insert remote prcoess group details."); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - - private void createConnectDetails(int actionId, ConnectDetails connectionDetails) throws DataAccessException { - PreparedStatement statement = null; - try { - // obtain a statement to insert to the processor action table - statement = connection.prepareStatement(INSERT_CONNECT_DETAILS); - statement.setInt(1, actionId); - statement.setString(2, connectionDetails.getSourceId()); - statement.setString(3, StringUtils.left(connectionDetails.getSourceName(), 1000)); - statement.setString(4, StringUtils.left(connectionDetails.getSourceType().toString(), 1000)); - statement.setString(5, StringUtils.left(connectionDetails.getRelationship(), 1000)); - statement.setString(6, connectionDetails.getDestinationId()); - statement.setString(7, StringUtils.left(connectionDetails.getDestinationName(), 1000)); - statement.setString(8, StringUtils.left(connectionDetails.getDestinationType().toString(), 1000)); - - // insert the action - int updateCount = statement.executeUpdate(); - - // ensure the operation completed successfully - if (updateCount != 1) { - throw new DataAccessException("Unable to insert connection details."); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - - private void createMoveDetails(int actionId, MoveDetails moveDetails) throws DataAccessException { - PreparedStatement statement = null; - try { - // obtain a statement to insert to the processor action table - statement = connection.prepareStatement(INSERT_MOVE_DETAILS); - statement.setInt(1, actionId); - statement.setString(2, moveDetails.getGroupId()); - statement.setString(3, StringUtils.left(moveDetails.getGroup(), 1000)); - statement.setString(4, moveDetails.getPreviousGroupId()); - statement.setString(5, StringUtils.left(moveDetails.getPreviousGroup(), 1000)); - - // insert the action - int updateCount = statement.executeUpdate(); - - // ensure the operation completed successfully - if (updateCount != 1) { - throw new DataAccessException("Unable to insert move details."); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - - private void createConfigureDetails(int actionId, ConfigureDetails configurationDetails) throws DataAccessException { - PreparedStatement statement = null; - try { - // obtain a statement to insert to the processor action table - statement = connection.prepareStatement(INSERT_CONFIGURE_DETAILS); - statement.setInt(1, actionId); - statement.setString(2, StringUtils.left(configurationDetails.getName(), 1000)); - statement.setString(3, StringUtils.left(configurationDetails.getValue(), 5000)); - statement.setString(4, StringUtils.left(configurationDetails.getPreviousValue(), 5000)); - - // insert the action - int updateCount = statement.executeUpdate(); - - // ensure the operation completed successfully - if (updateCount != 1) { - throw new DataAccessException("Unable to insert configure details."); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - - private void createPurgeDetails(int actionId, PurgeDetails purgeDetails) throws DataAccessException { - PreparedStatement statement = null; - try { - // obtain a statement to insert to the processor action table - statement = connection.prepareStatement(INSERT_PURGE_DETAILS); - statement.setInt(1, actionId); - statement.setTimestamp(2, new java.sql.Timestamp(purgeDetails.getEndDate().getTime())); - - // insert the action - int updateCount = statement.executeUpdate(); - - // ensure the operation completed successfully - if (updateCount != 1) { - throw new DataAccessException("Unable to insert connection details."); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - - @Override - public History findActions(HistoryQuery historyQuery) throws DataAccessException { - // get the sort column - String sortColumn = ACTION_TIMESTAMP; - if (StringUtils.isNotBlank(historyQuery.getSortColumn())) { - String rawColumnName = historyQuery.getSortColumn(); - if (!columnMap.containsKey(rawColumnName)) { - throw new IllegalArgumentException(String.format("Unrecognized column name '%s'.", rawColumnName)); - } - sortColumn = columnMap.get(rawColumnName); - } - - // get the sort order - String sortOrder = StringUtils.defaultIfBlank(historyQuery.getSortOrder(), "desc"); - - History actionResult = new History(); - Collection actions = new ArrayList<>(); - PreparedStatement statement = null; - ResultSet rs = null; - try { - List where = new ArrayList<>(); - - // append the start time - if (historyQuery.getStartDate() != null) { - where.add("ACTION_TIMESTAMP >= ?"); - } - - // append the end time - if (historyQuery.getEndDate() != null) { - where.add("ACTION_TIMESTAMP <= ?"); - } - - // append the user id as necessary - if (historyQuery.getUserIdentity() != null) { - where.add("UPPER(IDENTITY) LIKE ?"); - } - - // append the source id as necessary - if (historyQuery.getSourceId() != null) { - where.add("SOURCE_ID = ?"); - } - - String sql = SELECT_ACTION_COUNT; - if (!where.isEmpty()) { - sql += " WHERE " + StringUtils.join(where, " AND "); - } - - // get the total number of actions - statement = connection.prepareStatement(sql); - int paramIndex = 1; - - // set the start date as necessary - if (historyQuery.getStartDate() != null) { - statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getStartDate().getTime())); - } - - // set the end date as necessary - if (historyQuery.getEndDate() != null) { - statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getEndDate().getTime())); - } - - // set the user id as necessary - if (historyQuery.getUserIdentity() != null) { - statement.setString(paramIndex++, "%" + historyQuery.getUserIdentity().toUpperCase() + "%"); - } - - // set the source id as necessary - if (historyQuery.getSourceId() != null) { - statement.setString(paramIndex, historyQuery.getSourceId()); - } - - // execute the statement - rs = statement.executeQuery(); - - // ensure there are results - if (rs.next()) { - actionResult.setTotal(rs.getInt("ACTION_COUNT")); - } else { - throw new DataAccessException("Unable to determine total action count."); - } - - sql = SELECT_ACTIONS; - if (!where.isEmpty()) { - sql += " WHERE " + StringUtils.join(where, " AND "); - } - - // append the sort criteria - sql += (" ORDER BY " + sortColumn + " " + sortOrder); - - // append the offset and limit - sql += " LIMIT ? OFFSET ?"; - - // close the previous statement - statement.close(); - - // create the statement - statement = connection.prepareStatement(sql); - paramIndex = 1; - - // set the start date as necessary - if (historyQuery.getStartDate() != null) { - statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getStartDate().getTime())); - } - - // set the end date as necessary - if (historyQuery.getEndDate() != null) { - statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getEndDate().getTime())); - } - - // set the user id as necessary - if (historyQuery.getUserIdentity() != null) { - statement.setString(paramIndex++, "%" + historyQuery.getUserIdentity().toUpperCase() + "%"); - } - - // set the source id as necessary - if (historyQuery.getSourceId() != null) { - statement.setString(paramIndex++, historyQuery.getSourceId()); - } - - // set the limit - statement.setInt(paramIndex++, historyQuery.getCount()); - - // set the offset according to the currented page calculated above - statement.setInt(paramIndex, historyQuery.getOffset()); - - // execute the query - rs = statement.executeQuery(); - - // create each corresponding action - while (rs.next()) { - final Integer actionId = rs.getInt("ID"); - final Operation operation = Operation.valueOf(rs.getString(OPERATION)); - final Component component = Component.valueOf(rs.getString(SOURCE_TYPE)); - - FlowChangeAction action = new FlowChangeAction(); - action.setId(actionId); - action.setUserIdentity(rs.getString(IDENTITY)); - action.setOperation(Operation.valueOf(rs.getString(OPERATION))); - action.setTimestamp(new Date(rs.getTimestamp(ACTION_TIMESTAMP).getTime())); - action.setSourceId(rs.getString(SOURCE_ID)); - action.setSourceName(rs.getString(SOURCE_NAME)); - action.setSourceType(Component.valueOf(rs.getString(SOURCE_TYPE))); - - // get the component details if appropriate - ComponentDetails componentDetails = null; - if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component) - || Component.ParameterProvider.equals(component)) { - componentDetails = getExtensionDetails(actionId); - } else if (Component.RemoteProcessGroup.equals(component)) { - componentDetails = getRemoteProcessGroupDetails(actionId); - } - - if (componentDetails != null) { - action.setComponentDetails(componentDetails); - } - - // get the action details if appropriate - ActionDetails actionDetails = null; - if (Operation.Move.equals(operation)) { - actionDetails = getMoveDetails(actionId); - } else if (Operation.Configure.equals(operation)) { - actionDetails = getConfigureDetails(actionId); - } else if (Operation.Connect.equals(operation) || Operation.Disconnect.equals(operation)) { - actionDetails = getConnectDetails(actionId); - } else if (Operation.Purge.equals(operation)) { - actionDetails = getPurgeDetails(actionId); - } - - // set the action details - if (actionDetails != null) { - action.setActionDetails(actionDetails); - } - - // add the action - actions.add(action); - } - - // populate the action result - actionResult.setActions(actions); - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return actionResult; - } - - @Override - public Action getAction(Integer actionId) throws DataAccessException { - FlowChangeAction action = null; - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_ACTION_BY_ID); - statement.setInt(1, actionId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - if (rs.next()) { - Operation operation = Operation.valueOf(rs.getString(OPERATION)); - Component component = Component.valueOf(rs.getString(SOURCE_TYPE)); - - // populate the action - action = new FlowChangeAction(); - action.setId(rs.getInt("ID")); - action.setUserIdentity(rs.getString(IDENTITY)); - action.setOperation(operation); - action.setTimestamp(new Date(rs.getTimestamp(ACTION_TIMESTAMP).getTime())); - action.setSourceId(rs.getString(SOURCE_ID)); - action.setSourceName(rs.getString(SOURCE_NAME)); - action.setSourceType(component); - - // get the component details if appropriate - ComponentDetails componentDetails = null; - if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component) - || Component.ParameterProvider.equals(component) || Component.FlowRegistryClient.equals(component)) { - componentDetails = getExtensionDetails(actionId); - } else if (Component.RemoteProcessGroup.equals(component)) { - componentDetails = getRemoteProcessGroupDetails(actionId); - } - - if (componentDetails != null) { - action.setComponentDetails(componentDetails); - } - - // get the action details if appropriate - ActionDetails actionDetails = null; - if (Operation.Move.equals(operation)) { - actionDetails = getMoveDetails(actionId); - } else if (Operation.Configure.equals(operation)) { - actionDetails = getConfigureDetails(actionId); - } else if (Operation.Connect.equals(operation) || Operation.Disconnect.equals(operation)) { - actionDetails = getConnectDetails(actionId); - } else if (Operation.Purge.equals(operation)) { - actionDetails = getPurgeDetails(actionId); - } - - // set the action details - if (actionDetails != null) { - action.setActionDetails(actionDetails); - } - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return action; - } - - private ExtensionDetails getExtensionDetails(Integer actionId) throws DataAccessException { - FlowChangeExtensionDetails extensionDetails = null; - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_EXTENSION_DETAILS_FOR_ACTION); - statement.setInt(1, actionId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - if (rs.next()) { - extensionDetails = new FlowChangeExtensionDetails(); - extensionDetails.setType(rs.getString("TYPE")); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return extensionDetails; - } - - private RemoteProcessGroupDetails getRemoteProcessGroupDetails(Integer actionId) throws DataAccessException { - FlowChangeRemoteProcessGroupDetails remoteProcessGroupDetails = null; - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION); - statement.setInt(1, actionId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - if (rs.next()) { - remoteProcessGroupDetails = new FlowChangeRemoteProcessGroupDetails(); - remoteProcessGroupDetails.setUri(rs.getString("URI")); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return remoteProcessGroupDetails; - } - - private MoveDetails getMoveDetails(Integer actionId) throws DataAccessException { - FlowChangeMoveDetails moveDetails = null; - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_MOVE_DETAILS_FOR_ACTION); - statement.setInt(1, actionId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - if (rs.next()) { - moveDetails = new FlowChangeMoveDetails(); - moveDetails.setGroupId(rs.getString("GROUP_ID")); - moveDetails.setGroup(rs.getString("GROUP_NAME")); - moveDetails.setPreviousGroupId(rs.getString("PREVIOUS_GROUP_ID")); - moveDetails.setPreviousGroup(rs.getString("PREVIOUS_GROUP_NAME")); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return moveDetails; - } - - private ConnectDetails getConnectDetails(Integer actionId) throws DataAccessException { - FlowChangeConnectDetails connectionDetails = null; - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_CONNECT_DETAILS_FOR_ACTION); - statement.setInt(1, actionId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - if (rs.next()) { - final Component sourceComponent = Component.valueOf(rs.getString(SOURCE_TYPE)); - final Component destinationComponent = Component.valueOf(rs.getString("DESTINATION_TYPE")); - - connectionDetails = new FlowChangeConnectDetails(); - connectionDetails.setSourceId(rs.getString(SOURCE_ID)); - connectionDetails.setSourceName(rs.getString(SOURCE_NAME)); - connectionDetails.setSourceType(sourceComponent); - connectionDetails.setRelationship(rs.getString("RELATIONSHIP")); - connectionDetails.setDestinationId(rs.getString("DESTINATION_ID")); - connectionDetails.setDestinationName(rs.getString("DESTINATION_NAME")); - connectionDetails.setDestinationType(destinationComponent); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return connectionDetails; - } - - private ConfigureDetails getConfigureDetails(Integer actionId) throws DataAccessException { - FlowChangeConfigureDetails configurationDetails = null; - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_CONFIGURE_DETAILS_FOR_ACTION); - statement.setInt(1, actionId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - if (rs.next()) { - configurationDetails = new FlowChangeConfigureDetails(); - configurationDetails.setName(rs.getString("NAME")); - configurationDetails.setValue(rs.getString("VALUE")); - configurationDetails.setPreviousValue(rs.getString("PREVIOUS_VALUE")); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return configurationDetails; - } - - private PurgeDetails getPurgeDetails(Integer actionId) throws DataAccessException { - FlowChangePurgeDetails purgeDetails = null; - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_PURGE_DETAILS_FOR_ACTION); - statement.setInt(1, actionId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - if (rs.next()) { - purgeDetails = new FlowChangePurgeDetails(); - purgeDetails.setEndDate(new Date(rs.getTimestamp("END_DATE").getTime())); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return purgeDetails; - } - - @Override - public Map> getPreviousValues(String componentId) { - Map> previousValues = new LinkedHashMap<>(); - - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_PREVIOUSLY_CONFIGURED_FIELDS); - statement.setString(1, componentId); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - while (rs.next()) { - final String property = rs.getString("NAME"); - previousValues.put(property, getPreviousValuesForProperty(componentId, property)); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return previousValues; - } - - private List getPreviousValuesForProperty(final String componentId, final String property) { - List previousValues = new ArrayList<>(); - - PreparedStatement statement = null; - ResultSet rs = null; - try { - // create the statement - statement = connection.prepareStatement(SELECT_PREVIOUS_VALUES); - statement.setString(1, componentId); - statement.setString(2, property); - - // execute the query - rs = statement.executeQuery(); - - // ensure results - while (rs.next()) { - // get the previous value - final PreviousValue previousValue = new PreviousValue(); - previousValue.setPreviousValue(rs.getString("VALUE")); - previousValue.setTimestamp(new Date(rs.getTimestamp(ACTION_TIMESTAMP).getTime())); - previousValue.setUserIdentity(rs.getString(IDENTITY)); - previousValues.add(previousValue); - } - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(rs); - RepositoryUtils.closeQuietly(statement); - } - - return previousValues; - } - - @Override - public void deleteActions(Date endDate) throws DataAccessException { - PreparedStatement statement = null; - try { - // ----------------- - // component details - // ----------------- - - // create the move delete statement - statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "PROCESSOR_DETAILS", ACTION_ID)); - statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); - statement.executeUpdate(); - statement.close(); - - // create the move delete statement - statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "REMOTE_PROCESS_GROUP_DETAILS", ACTION_ID)); - statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); - statement.executeUpdate(); - statement.close(); - - // -------------- - // action details - // -------------- - // create the move delete statement - statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "MOVE_DETAILS", ACTION_ID)); - statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); - statement.executeUpdate(); - statement.close(); - - // create the configure delete statement - statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "CONFIGURE_DETAILS", ACTION_ID)); - statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); - statement.executeUpdate(); - statement.close(); - - // create the connect delete statement - statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "CONNECT_DETAILS", ACTION_ID)); - statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); - statement.executeUpdate(); - statement.close(); - - // create the relationship delete statement - statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "PURGE_DETAILS", ACTION_ID)); - statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); - statement.executeUpdate(); - statement.close(); - - // ------- - // actions - // ------- - // create the action delete statement - statement = connection.prepareStatement(DELETE_ACTIONS); - statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); - statement.executeUpdate(); - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - - @Override - public void deletePreviousValues(String propertyName, String componentId) { - PreparedStatement statement = null; - try { - statement = connection.prepareStatement(DELETE_PREVIOUS_VALUES); - statement.setString(1, propertyName); - statement.setString(2, componentId); - statement.executeUpdate(); - statement.close(); - } catch (SQLException sqle) { - throw new DataAccessException(sqle); - } finally { - RepositoryUtils.closeQuietly(statement); - } - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/EntityStoreAuditService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/EntityStoreAuditService.java new file mode 100644 index 0000000000..a978e1238d --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/EntityStoreAuditService.java @@ -0,0 +1,636 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service; + +import jetbrains.exodus.entitystore.Entity; +import jetbrains.exodus.entitystore.EntityId; +import jetbrains.exodus.entitystore.EntityIterable; +import jetbrains.exodus.entitystore.PersistentEntityStore; +import jetbrains.exodus.entitystore.PersistentEntityStores; +import jetbrains.exodus.entitystore.StoreTransaction; +import jetbrains.exodus.env.Environment; +import jetbrains.exodus.env.EnvironmentConfig; +import jetbrains.exodus.env.Environments; +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.FlowChangeAction; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ComponentDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; +import org.apache.nifi.action.component.details.FlowChangeExtensionDetails; +import org.apache.nifi.action.component.details.FlowChangeRemoteProcessGroupDetails; +import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.action.details.ConnectDetails; +import org.apache.nifi.action.details.FlowChangeConfigureDetails; +import org.apache.nifi.action.details.FlowChangeConnectDetails; +import org.apache.nifi.action.details.FlowChangeMoveDetails; +import org.apache.nifi.action.details.FlowChangePurgeDetails; +import org.apache.nifi.action.details.MoveDetails; +import org.apache.nifi.action.details.PurgeDetails; +import org.apache.nifi.admin.service.entity.ActionEntity; +import org.apache.nifi.admin.service.entity.ActionLink; +import org.apache.nifi.admin.service.entity.ConfigureDetailsEntity; +import org.apache.nifi.admin.service.entity.ConnectDetailsEntity; +import org.apache.nifi.admin.service.entity.EntityProperty; +import org.apache.nifi.admin.service.entity.EntityType; +import org.apache.nifi.admin.service.entity.MoveDetailsEntity; +import org.apache.nifi.admin.service.entity.PurgeDetailsEntity; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.PreviousValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Stream; + +/** + * Audit Service implementation based on JetBrains Xodus Entity Store + */ +public class EntityStoreAuditService implements AuditService, Closeable { + private static final long FIRST_START_TIME = 0; + + private static final int PREVIOUS_VALUES_LIMIT = 5; + + private static final String ASCENDING_SORT_ORDER = "ASC"; + + private static final int DEFAULT_COUNT = 100; + + private static final String BACKUP_FILE_NAME_FORMAT = "%s.backup.%d"; + + private static final Logger logger = LoggerFactory.getLogger(EntityStoreAuditService.class); + + private final PersistentEntityStore entityStore; + + private final Environment environment; + + /** + * Entity Store Audit Service constructor with required properties for persistent location + * + * @param directory Persistent Entity Store directory + */ + public EntityStoreAuditService(final File directory) { + environment = loadEnvironment(directory); + entityStore = PersistentEntityStores.newInstance(environment); + logger.info("Environment configured with directory [{}]", directory); + } + + /** + * Add Actions to Persistent Store + * + * @param actions Collections of Actions to be added + */ + @Override + public void addActions(final Collection actions) { + Objects.requireNonNull(actions, "Actions required"); + + entityStore.executeInExclusiveTransaction(storeTransaction -> { + for (final Action action : actions) { + addAction(storeTransaction, action); + } + logger.debug("Actions added [{}]", actions.size()); + }); + } + + /** + * Get Previous Values for specified Component Identifier + * + * @param componentId Component Identifier for which previous property values should be retrieved + * @return Map of Property Name to List of Previous Property Values + */ + @Override + public Map> getPreviousValues(final String componentId) { + Objects.requireNonNull(componentId, "Component Identifier required"); + + return entityStore.computeInReadonlyTransaction(storeTransaction -> { + final Map> previousValuesFound = new LinkedHashMap<>(); + + final EntityIterable actionEntities = storeTransaction.find(EntityType.ACTION.getEntityType(), ActionEntity.SOURCE_ID.getProperty(), componentId); + for (Entity actionEntity : actionEntities) { + final Entity configureDetails = actionEntity.getLink(ActionLink.CONFIGURE_DETAILS.getProperty()); + if (configureDetails != null) { + final String name = getProperty(configureDetails, ConfigureDetailsEntity.NAME); + final String value = getProperty(configureDetails, ConfigureDetailsEntity.VALUE); + + final PreviousValue previousValue = new PreviousValue(); + previousValue.setPreviousValue(value); + previousValue.setUserIdentity(getProperty(actionEntity, ActionEntity.USER_IDENTITY)); + previousValue.setTimestamp(getDateProperty(actionEntity, ActionEntity.TIMESTAMP)); + + final List previousValues = previousValuesFound.get(name); + if (previousValues == null) { + final List newPreviousValues = new ArrayList<>(); + newPreviousValues.add(previousValue); + previousValuesFound.put(name, newPreviousValues); + } else if (previousValues.size() < PREVIOUS_VALUES_LIMIT) { + previousValues.add(previousValue); + } + } + } + + return previousValuesFound; + }); + } + + /** + * Delete Previous Values for specified Component Identifier and Property + * + * @param propertyName Name of the property for which Previous Values should be deleted + * @param componentId Component Identifier for which Previous Values should be deleted + */ + @Override + public void deletePreviousValues(final String propertyName, final String componentId) { + Objects.requireNonNull(propertyName, "Property Name required"); + Objects.requireNonNull(componentId, "Component Identifier required"); + + entityStore.executeInExclusiveTransaction(storeTransaction -> { + final EntityIterable actionEntities = storeTransaction.find(EntityType.ACTION.getEntityType(), ActionEntity.SOURCE_ID.getProperty(), componentId); + for (Entity actionEntity : actionEntities) { + final Entity configureDetails = actionEntity.getLink(ActionLink.CONFIGURE_DETAILS.getProperty()); + if (configureDetails != null) { + final Comparable configureDetailsName = configureDetails.getProperty(ConfigureDetailsEntity.NAME.getProperty()); + if (propertyName.equals(configureDetailsName)) { + actionEntity.deleteLinks(ActionLink.CONFIGURE_DETAILS.getProperty()); + } + } + } + }); + + logger.info("Component [{}] Previous Property Values deleted", componentId); + } + + /** + * Get Actions based on query parameters + * + * @param actionQuery History Action Query + * @return Actions found + */ + @Override + public History getActions(final HistoryQuery actionQuery) { + Objects.requireNonNull(actionQuery, "History Query required"); + + return entityStore.computeInReadonlyTransaction(storeTransaction -> { + final Collection actionsFound = new ArrayList<>(); + final EntityIterable entities = findActionEntities(actionQuery, storeTransaction); + final int total = Math.toIntExact(entities.size()); + + final Integer queryOffset = actionQuery.getOffset(); + final int skip = Objects.requireNonNullElse(queryOffset, 0); + + final Integer queryCount = actionQuery.getCount(); + final int limit = Objects.requireNonNullElse(queryCount, DEFAULT_COUNT); + + final EntityIterable selected = entities.skip(skip).take(limit); + for (final Entity entity : selected) { + final Action action = readAction(entity); + actionsFound.add(action); + } + + final History history = new History(); + history.setActions(actionsFound); + history.setTotal(total); + history.setLastRefreshed(new Date()); + return history; + }); + } + + /** + * Get Actions starting with indicated Action Identifier + * + * @param firstActionId First Action Identifier + * @param maxActions Maximum number of Actions to be returned + * @return Actions found + */ + @Override + public History getActions(final int firstActionId, final int maxActions) { + return entityStore.computeInReadonlyTransaction(storeTransaction -> { + final Collection actions = new ArrayList<>(); + + final int lastActionId = firstActionId + maxActions; + final EntityIterable found = storeTransaction.findIds(EntityType.ACTION.getEntityType(), firstActionId, lastActionId); + + for (final Entity entity : found) { + final Action action = readAction(entity); + actions.add(action); + } + + final History history = new History(); + history.setActions(actions); + history.setLastRefreshed(new Date()); + history.setTotal(actions.size()); + + return history; + }); + } + + /** + * Get Action for specified identifier + * + * @param actionId Action Identifier + * @return Action or null when not found + */ + @Override + public Action getAction(final Integer actionId) { + Objects.requireNonNull(actionId, "Action Identifier required"); + + return entityStore.computeInReadonlyTransaction(storeTransaction -> { + final Action action; + + final EntityIterable found = storeTransaction.findIds(EntityType.ACTION.getEntityType(), actionId, actionId); + final Entity entity = found.getFirst(); + if (entity == null) { + action = null; + } else { + action = readAction(entity); + } + + return action; + }); + } + + /** + * Purge Actions from Persistent Entity Store clearing existing records and creating Purge Action + * + * @param end End time for purge action query + * @param purgeAction Purge Action to be recorded + */ + @Override + public void purgeActions(final Date end, final Action purgeAction) { + Objects.requireNonNull(end, "End date required"); + Objects.requireNonNull(purgeAction, "Purge Action required"); + + final long endTime = end.getTime(); + entityStore.executeInExclusiveTransaction(storeTransaction -> { + final EntityIterable entities = storeTransaction.find(EntityType.ACTION.getEntityType(), ActionEntity.TIMESTAMP.getProperty(), FIRST_START_TIME, endTime); + for (final Entity entity : entities) { + entity.delete(); + + for (final ActionLink actionLink : ActionLink.values()) { + entity.deleteLinks(actionLink.getProperty()); + } + } + + addAction(storeTransaction, purgeAction); + }); + + logger.info("User [{}] Purged Actions with end date [{}]", purgeAction.getUserIdentity(), end); + } + + /** + * Close Persistent Entity Store resources + * + * @throws IOException Thrown on failures to close Entity Store + */ + @Override + public void close() throws IOException { + entityStore.close(); + environment.close(); + logger.info("Environment closed"); + } + + private EntityIterable findActionEntities(final HistoryQuery actionQuery, final StoreTransaction storeTransaction) { + final long startTimestamp; + final long endTimestamp; + + final Date startDate = actionQuery.getStartDate(); + if (startDate == null) { + startTimestamp = FIRST_START_TIME; + } else { + startTimestamp = startDate.getTime(); + } + + final Date endDate = actionQuery.getEndDate(); + if (endDate == null) { + endTimestamp = System.currentTimeMillis(); + } else { + endTimestamp = endDate.getTime(); + } + + final ActionEntity sortEntityProperty = getSortEntityProperty(actionQuery); + final boolean ascending = isAscending(actionQuery); + final EntityIterable sorted = storeTransaction.sort(EntityType.ACTION.getEntityType(), sortEntityProperty.getProperty(), ascending); + final EntityIterable entities = sorted.intersect(storeTransaction.find(EntityType.ACTION.getEntityType(), ActionEntity.TIMESTAMP.getProperty(), startTimestamp, endTimestamp)); + + final EntityIterable sourceEntities; + final String sourceId = actionQuery.getSourceId(); + if (sourceId == null) { + sourceEntities = entities; + } else { + final EntityIterable sourceFiltered = storeTransaction.find(EntityType.ACTION.getEntityType(), ActionEntity.SOURCE_ID.getProperty(), sourceId); + sourceEntities = entities.intersect(sourceFiltered); + } + + final EntityIterable filteredEntities; + final String userIdentity = actionQuery.getUserIdentity(); + if (userIdentity == null) { + filteredEntities = sourceEntities; + } else { + final EntityIterable identityFiltered = storeTransaction.find(EntityType.ACTION.getEntityType(), ActionEntity.USER_IDENTITY.getProperty(), userIdentity); + filteredEntities = sourceEntities.intersect(identityFiltered); + } + + return filteredEntities; + } + + private boolean isAscending(final HistoryQuery historyQuery) { + final String sortOrder = historyQuery.getSortOrder(); + + final boolean ascending; + if (sortOrder == null || sortOrder.isEmpty()) { + ascending = false; + } else { + ascending = ASCENDING_SORT_ORDER.equalsIgnoreCase(sortOrder); + } + + return ascending; + } + + private ActionEntity getSortEntityProperty(final HistoryQuery historyQuery) { + final String sortColumn = historyQuery.getSortColumn(); + + final ActionEntity sortEntityProperty; + + if (sortColumn == null || sortColumn.isEmpty()) { + sortEntityProperty = ActionEntity.TIMESTAMP; + } else { + ActionEntity foundActionEntity = null; + for (final ActionEntity actionEntity : ActionEntity.values()) { + if (actionEntity.getProperty().equals(sortColumn)) { + foundActionEntity = actionEntity; + break; + } + } + + if (foundActionEntity == null) { + throw new IllegalArgumentException("Specified Sort Column not supported"); + } else { + sortEntityProperty = foundActionEntity; + } + } + + + return sortEntityProperty; + } + + private void addAction(final StoreTransaction storeTransaction, final Action action) { + final Entity actionEntity = storeTransaction.newEntity(EntityType.ACTION.getEntityType()); + actionEntity.setProperty(ActionEntity.TIMESTAMP.getProperty(), action.getTimestamp().getTime()); + actionEntity.setProperty(ActionEntity.USER_IDENTITY.getProperty(), action.getUserIdentity()); + actionEntity.setProperty(ActionEntity.SOURCE_ID.getProperty(), action.getSourceId()); + actionEntity.setProperty(ActionEntity.SOURCE_NAME.getProperty(), action.getSourceName()); + actionEntity.setProperty(ActionEntity.SOURCE_TYPE.getProperty(), action.getSourceType().name()); + actionEntity.setProperty(ActionEntity.OPERATION.getProperty(), action.getOperation().name()); + + final ComponentDetails componentDetails = action.getComponentDetails(); + addComponentDetails(actionEntity, componentDetails); + + final ActionDetails actionDetails = action.getActionDetails(); + addActionDetails(storeTransaction, actionEntity, actionDetails); + } + + private void addComponentDetails(final Entity actionEntity, final ComponentDetails componentDetails) { + if (componentDetails instanceof ExtensionDetails extensionDetails) { + actionEntity.setProperty(ActionEntity.EXTENSION_TYPE.getProperty(), extensionDetails.getType()); + } else if (componentDetails instanceof RemoteProcessGroupDetails remoteProcessGroupDetails) { + actionEntity.setProperty(ActionEntity.REMOTE_PROCESS_GROUP_URI.getProperty(), remoteProcessGroupDetails.getUri()); + } + } + + private void addActionDetails(final StoreTransaction storeTransaction, final Entity actionEntity, final ActionDetails actionDetails) { + if (actionDetails instanceof ConnectDetails connectDetails) { + addConnectDetails(storeTransaction, actionEntity, connectDetails); + } else if (actionDetails instanceof MoveDetails moveDetails) { + addMoveDetails(storeTransaction, actionEntity, moveDetails); + } else if (actionDetails instanceof ConfigureDetails configureDetails) { + addConfigureDetails(storeTransaction, actionEntity, configureDetails); + } else if (actionDetails instanceof PurgeDetails purgeDetails) { + addPurgeDetails(storeTransaction, actionEntity, purgeDetails); + } + } + + private void addConnectDetails(final StoreTransaction storeTransaction, final Entity actionEntity, final ConnectDetails connectDetails) { + final Entity connectDetailsEntity = storeTransaction.newEntity(EntityType.CONNECT_DETAILS.getEntityType()); + connectDetailsEntity.setLink(ConnectDetailsEntity.ACTION.getProperty(), actionEntity); + actionEntity.setLink(ActionLink.CONNECT_DETAILS.getProperty(), connectDetailsEntity); + + connectDetailsEntity.setProperty(ConnectDetailsEntity.SOURCE_ID.getProperty(), connectDetails.getSourceId()); + connectDetailsEntity.setProperty(ConnectDetailsEntity.SOURCE_NAME.getProperty(), connectDetails.getSourceName()); + connectDetailsEntity.setProperty(ConnectDetailsEntity.SOURCE_TYPE.getProperty(), connectDetails.getSourceType().name()); + connectDetailsEntity.setProperty(ConnectDetailsEntity.DESTINATION_ID.getProperty(), connectDetails.getDestinationId()); + connectDetailsEntity.setProperty(ConnectDetailsEntity.DESTINATION_NAME.getProperty(), connectDetails.getDestinationName()); + connectDetailsEntity.setProperty(ConnectDetailsEntity.DESTINATION_TYPE.getProperty(), connectDetails.getDestinationType().name()); + connectDetailsEntity.setProperty(ConnectDetailsEntity.RELATIONSHIP.getProperty(), connectDetails.getRelationship()); + } + + private void addMoveDetails(final StoreTransaction storeTransaction, final Entity actionEntity, final MoveDetails moveDetails) { + final Entity moveDetailsEntity = storeTransaction.newEntity(EntityType.MOVE_DETAILS.getEntityType()); + moveDetailsEntity.setLink(MoveDetailsEntity.ACTION.getProperty(), actionEntity); + actionEntity.setLink(ActionLink.MOVE_DETAILS.getProperty(), moveDetailsEntity); + + moveDetailsEntity.setProperty(MoveDetailsEntity.GROUP.getProperty(), moveDetails.getGroup()); + moveDetailsEntity.setProperty(MoveDetailsEntity.GROUP_ID.getProperty(), moveDetails.getGroupId()); + moveDetailsEntity.setProperty(MoveDetailsEntity.PREVIOUS_GROUP.getProperty(), moveDetails.getPreviousGroup()); + moveDetailsEntity.setProperty(MoveDetailsEntity.PREVIOUS_GROUP_ID.getProperty(), moveDetails.getPreviousGroupId()); + } + + private void addConfigureDetails(final StoreTransaction storeTransaction, final Entity actionEntity, final ConfigureDetails configureDetails) { + final Entity configureDetailsEntity = storeTransaction.newEntity(EntityType.CONFIGURE_DETAILS.getEntityType()); + configureDetailsEntity.setLink(MoveDetailsEntity.ACTION.getProperty(), actionEntity); + actionEntity.setLink(ActionLink.CONFIGURE_DETAILS.getProperty(), configureDetailsEntity); + + configureDetailsEntity.setProperty(ConfigureDetailsEntity.NAME.getProperty(), configureDetails.getName()); + + final String previousValue = configureDetails.getPreviousValue(); + if (previousValue != null) { + configureDetailsEntity.setProperty(ConfigureDetailsEntity.PREVIOUS_VALUE.getProperty(), previousValue); + } + + final String value = configureDetails.getValue(); + if (value != null) { + configureDetailsEntity.setProperty(ConfigureDetailsEntity.VALUE.getProperty(), value); + } + } + + private void addPurgeDetails(final StoreTransaction storeTransaction, final Entity actionEntity, final PurgeDetails purgeDetails) { + final Entity purgeDetailsEntity = storeTransaction.newEntity(EntityType.PURGE_DETAILS.getEntityType()); + purgeDetailsEntity.setLink(PurgeDetailsEntity.ACTION.getProperty(), actionEntity); + actionEntity.setLink(ActionLink.PURGE_DETAILS.getProperty(), purgeDetailsEntity); + + purgeDetailsEntity.setProperty(PurgeDetailsEntity.END_DATE.getProperty(), purgeDetails.getEndDate().getTime()); + } + + private Action readAction(final Entity entity) { + final FlowChangeAction action = new FlowChangeAction(); + + final EntityId entityId = entity.getId(); + final int id = Math.toIntExact(entityId.getLocalId()); + action.setId(id); + + action.setUserIdentity(getProperty(entity, ActionEntity.USER_IDENTITY)); + action.setSourceId(getProperty(entity, ActionEntity.SOURCE_ID)); + action.setSourceName(getProperty(entity, ActionEntity.SOURCE_NAME)); + action.setTimestamp(getDateProperty(entity, ActionEntity.TIMESTAMP)); + action.setSourceType(getEnumProperty(entity, ActionEntity.SOURCE_TYPE, Component.class)); + action.setOperation(getEnumProperty(entity, ActionEntity.OPERATION, Operation.class)); + + final String extensionType = getProperty(entity, ActionEntity.EXTENSION_TYPE); + if (extensionType != null) { + final FlowChangeExtensionDetails extensionDetails = new FlowChangeExtensionDetails(); + extensionDetails.setType(extensionType); + action.setComponentDetails(extensionDetails); + } + + final String remoteProgressGroupUri = getProperty(entity, ActionEntity.REMOTE_PROCESS_GROUP_URI); + if (remoteProgressGroupUri != null) { + final FlowChangeRemoteProcessGroupDetails remoteProcessGroupDetails = new FlowChangeRemoteProcessGroupDetails(); + remoteProcessGroupDetails.setUri(remoteProgressGroupUri); + action.setComponentDetails(remoteProcessGroupDetails); + } + + final Entity purgeDetailsEntity = entity.getLink(ActionLink.PURGE_DETAILS.getProperty()); + if (purgeDetailsEntity != null) { + final FlowChangePurgeDetails purgeDetails = new FlowChangePurgeDetails(); + purgeDetails.setEndDate(getDateProperty(purgeDetailsEntity, PurgeDetailsEntity.END_DATE)); + action.setActionDetails(purgeDetails); + } + final Entity configureDetailsEntity = entity.getLink(ActionLink.CONFIGURE_DETAILS.getProperty()); + if (configureDetailsEntity != null) { + final ConfigureDetails configureDetails = getConfigureDetails(configureDetailsEntity); + action.setActionDetails(configureDetails); + } + final Entity connectDetailsEntity = entity.getLink(ActionLink.CONNECT_DETAILS.getProperty()); + if (connectDetailsEntity != null) { + final ConnectDetails connectDetails = getConnectDetails(connectDetailsEntity); + action.setActionDetails(connectDetails); + } + final Entity moveDetailsEntity = entity.getLink(ActionLink.MOVE_DETAILS.getProperty()); + if (moveDetailsEntity != null) { + final MoveDetails moveDetails = getMoveDetails(moveDetailsEntity); + action.setActionDetails(moveDetails); + } + + return action; + } + + private ConfigureDetails getConfigureDetails(final Entity configureDetailsEntity) { + final FlowChangeConfigureDetails configureDetails = new FlowChangeConfigureDetails(); + + configureDetails.setName(getProperty(configureDetailsEntity, ConfigureDetailsEntity.NAME)); + configureDetails.setPreviousValue(getProperty(configureDetailsEntity, ConfigureDetailsEntity.PREVIOUS_VALUE)); + configureDetails.setValue(getProperty(configureDetailsEntity, ConfigureDetailsEntity.VALUE)); + + return configureDetails; + } + + private ConnectDetails getConnectDetails(final Entity connectDetailsEntity) { + final FlowChangeConnectDetails connectDetails = new FlowChangeConnectDetails(); + + connectDetails.setSourceId(getProperty(connectDetailsEntity, ConnectDetailsEntity.SOURCE_ID)); + connectDetails.setSourceName(getProperty(connectDetailsEntity, ConnectDetailsEntity.SOURCE_NAME)); + connectDetails.setSourceType(getEnumProperty(connectDetailsEntity, ConnectDetailsEntity.SOURCE_TYPE, Component.class)); + connectDetails.setDestinationId(getProperty(connectDetailsEntity, ConnectDetailsEntity.DESTINATION_ID)); + connectDetails.setDestinationName(getProperty(connectDetailsEntity, ConnectDetailsEntity.DESTINATION_NAME)); + connectDetails.setDestinationType(getEnumProperty(connectDetailsEntity, ConnectDetailsEntity.DESTINATION_TYPE, Component.class)); + connectDetails.setRelationship(getProperty(connectDetailsEntity, ConnectDetailsEntity.RELATIONSHIP)); + + return connectDetails; + } + + private MoveDetails getMoveDetails(final Entity moveDetailsEntity) { + final FlowChangeMoveDetails moveDetails = new FlowChangeMoveDetails(); + + moveDetails.setGroup(getProperty(moveDetailsEntity, MoveDetailsEntity.GROUP)); + moveDetails.setGroupId(getProperty(moveDetailsEntity, MoveDetailsEntity.GROUP_ID)); + moveDetails.setPreviousGroup(getProperty(moveDetailsEntity, MoveDetailsEntity.PREVIOUS_GROUP)); + moveDetails.setPreviousGroupId(getProperty(moveDetailsEntity, MoveDetailsEntity.PREVIOUS_GROUP_ID)); + + return moveDetails; + } + + private String getProperty(final Entity entity, final EntityProperty entityProperty) { + final Comparable property = entity.getProperty(entityProperty.getProperty()); + return property == null ? null : property.toString(); + } + + private > T getEnumProperty(final Entity entity, final EntityProperty entityProperty, final Class enumType) { + final Comparable property = entity.getProperty(entityProperty.getProperty()); + return property == null ? null : Enum.valueOf(enumType, property.toString()); + } + + private Date getDateProperty(final Entity entity, final EntityProperty entityProperty) { + final Comparable property = entity.getProperty(entityProperty.getProperty()); + + final Date dateProperty; + + if (property instanceof Long) { + final long milliseconds = (Long) property; + dateProperty = new Date(milliseconds); + } else { + dateProperty = null; + } + + return dateProperty; + } + + private Environment loadEnvironment(final File directory) { + final EnvironmentConfig environmentConfig = new EnvironmentConfig(); + + Environment loadedEnvironment; + try { + loadedEnvironment = Environments.newInstance(directory, environmentConfig); + } catch (final Exception e) { + logger.warn("Environment loading failed with directory [{}]", directory, e); + + try (Stream files = Files.list(directory.toPath())) { + final List environmentFiles = files.filter(Files::isRegularFile).toList(); + final long now = System.currentTimeMillis(); + for (final Path environmentFile : environmentFiles) { + final String backupFileName = String.format(BACKUP_FILE_NAME_FORMAT, environmentFile.getFileName().toString(), now); + final Path backupStoreFile = environmentFile.resolveSibling(backupFileName); + try { + Files.move(environmentFile, backupStoreFile, StandardCopyOption.REPLACE_EXISTING); + logger.warn("Moved Environment file [{}] to [{}]", environmentFile, backupStoreFile); + } catch (final IOException ioe) { + throw new UncheckedIOException(String.format("Environment file move failed [%s]", environmentFile), ioe); + } + } + } catch (final IOException ioe) { + throw new UncheckedIOException(String.format("Environment directory listing failed [%s]", directory), ioe); + } + + // Retry loading after directory cleanup + loadedEnvironment = Environments.newInstance(directory, environmentConfig); + } + + return loadedEnvironment; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java deleted file mode 100644 index 937603e1ef..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.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.admin.service.action; - -import org.apache.nifi.action.Action; -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DAOFactory; - -import java.util.Collection; - -/** - * Adds the specified actions. - */ -public class AddActionsAction implements AdministrationAction { - - private final Collection actions; - - public AddActionsAction(Collection actions) { - this.actions = actions; - } - - @Override - public Void execute(DAOFactory daoFactory) { - ActionDAO actionDao = daoFactory.getActionDAO(); - - // add each action - for (Action action : actions) { - actionDao.createAction(action); - } - - return null; - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DeletePreviousValues.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DeletePreviousValues.java deleted file mode 100644 index 1c0e3a5e8d..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/DeletePreviousValues.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.admin.service.action; - -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DAOFactory; - -/** - * Purges actions up to a specified end date. - */ -public class DeletePreviousValues implements AdministrationAction { - - private final String propertyName; - private final String componentId; - - public DeletePreviousValues(String propertyName, String componentId) { - this.propertyName = propertyName; - this.componentId = componentId; - } - - @Override - public Void execute(DAOFactory daoFactory) { - ActionDAO actionDao = daoFactory.getActionDAO(); - - actionDao.deletePreviousValues(propertyName, componentId); - - return null; - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java deleted file mode 100644 index f975393b1f..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.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.admin.service.action; - -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DAOFactory; -import org.apache.nifi.history.History; -import org.apache.nifi.history.HistoryQuery; - -import java.util.Date; - -/** - * Get all actions that match the specified query. - */ -public class GetActionsAction implements AdministrationAction { - - private final HistoryQuery query; - - public GetActionsAction(HistoryQuery query) { - this.query = query; - } - - @Override - public History execute(DAOFactory daoFactory) { - ActionDAO actionDao = daoFactory.getActionDAO(); - - // find all matching history - History history = actionDao.findActions(query); - history.setLastRefreshed(new Date()); - - return history; - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java deleted file mode 100644 index 337643f226..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java +++ /dev/null @@ -1,43 +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.admin.service.action; - -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DAOFactory; -import org.apache.nifi.history.PreviousValue; - -import java.util.List; -import java.util.Map; - -/** - * Gets the action with the specified id. - */ -public class GetPreviousValues implements AdministrationAction>> { - - private final String componentId; - - public GetPreviousValues(String componentId) { - this.componentId = componentId; - } - - @Override - public Map> execute(DAOFactory daoFactory) { - ActionDAO actionDao = daoFactory.getActionDAO(); - return actionDao.getPreviousValues(componentId); - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java deleted file mode 100644 index 9d970dc766..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java +++ /dev/null @@ -1,51 +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.admin.service.action; - -import org.apache.nifi.action.Action; -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DAOFactory; - -import java.util.Date; - -/** - * Purges actions up to a specified end date. - */ -public class PurgeActionsAction implements AdministrationAction { - - private final Date end; - private final Action purgeAction; - - public PurgeActionsAction(Date end, Action purgeAction) { - this.end = end; - this.purgeAction = purgeAction; - } - - @Override - public Void execute(DAOFactory daoFactory) { - ActionDAO actionDao = daoFactory.getActionDAO(); - - // remove the corresponding actions - actionDao.deleteActions(end); - - // create a purge action - actionDao.createAction(purgeAction); - - return null; - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ActionEntity.java similarity index 57% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ActionEntity.java index 924e01f201..fcd3c9c080 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ActionEntity.java @@ -14,27 +14,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.admin.service.transaction; +package org.apache.nifi.admin.service.entity; /** - * Exception to indicate that the user account is disabled. + * Enumeration of Action properties stored as Entity objects */ -public class TransactionException extends RuntimeException { +public enum ActionEntity implements EntityProperty { + TIMESTAMP("timestamp"), - public TransactionException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); + USER_IDENTITY("userIdentity"), + + SOURCE_ID("sourceId"), + + SOURCE_NAME("sourceName"), + + SOURCE_TYPE("sourceType"), + + OPERATION("operation"), + + EXTENSION_TYPE("extensionType"), + + REMOTE_PROCESS_GROUP_URI("remoteProcessGroupUri"); + + private final String property; + + ActionEntity(final String property) { + this.property = property; } - public TransactionException(Throwable cause) { - super(cause); + @Override + public String getProperty() { + return property; } - - public TransactionException(String message, Throwable cause) { - super(message, cause); - } - - public TransactionException(String message) { - super(message); - } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ActionLink.java similarity index 61% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ActionLink.java index 28bfe2279d..5adc33e4c8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ActionLink.java @@ -14,27 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.admin.service.action; - -import org.apache.nifi.action.Action; -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DAOFactory; +package org.apache.nifi.admin.service.entity; /** - * Gets the action with the specified id. + * Enumeration of Action link properties for referencing other Entity objects */ -public class GetActionAction implements AdministrationAction { +public enum ActionLink implements EntityProperty { + CONNECT_DETAILS("connectDetails"), - private final Integer id; + MOVE_DETAILS("moveDetails"), - public GetActionAction(Integer id) { - this.id = id; + CONFIGURE_DETAILS("configureDetails"), + + PURGE_DETAILS("purgeDetails"); + + private final String property; + + ActionLink(final String property) { + this.property = property; } @Override - public Action execute(DAOFactory daoFactory) { - ActionDAO actionDao = daoFactory.getActionDAO(); - return actionDao.getAction(id); + public String getProperty() { + return property; } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ConfigureDetailsEntity.java similarity index 63% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ConfigureDetailsEntity.java index e4b71308ab..f370248a5e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ConfigureDetailsEntity.java @@ -14,34 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.user; - -import org.apache.nifi.authorization.user.NiFiUser; - -import java.util.Set; +package org.apache.nifi.admin.service.entity; /** - * + * Enumeration of Action Configure Details properties stored as Entity objects */ -public class NiFiUserGroup { +public enum ConfigureDetailsEntity implements EntityProperty { + ACTION("action"), - private String group; - private Set users; + NAME("name"), - public String getGroup() { - return group; + PREVIOUS_VALUE("previousValue"), + + VALUE("value"); + + private final String property; + + ConfigureDetailsEntity(final String property) { + this.property = property; } - public void setGroup(String group) { - this.group = group; + @Override + public String getProperty() { + return property; } - - public Set getUsers() { - return users; - } - - public void setUsers(Set users) { - this.users = users; - } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ConnectDetailsEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ConnectDetailsEntity.java new file mode 100644 index 0000000000..e4bc59acf2 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ConnectDetailsEntity.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.admin.service.entity; + +/** + * Enumeration of Action Connect Details properties stored as Entity objects + */ +public enum ConnectDetailsEntity implements EntityProperty { + ACTION("action"), + + SOURCE_ID("sourceId"), + + SOURCE_NAME("sourceName"), + + SOURCE_TYPE("sourceType"), + + DESTINATION_ID("destinationId"), + + DESTINATION_NAME("destinationName"), + + DESTINATION_TYPE("destinationType"), + + RELATIONSHIP("relationship"); + + private final String property; + + ConnectDetailsEntity(final String property) { + this.property = property; + } + + @Override + public String getProperty() { + return property; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/EntityProperty.java similarity index 75% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/EntityProperty.java index 2d2ef82eb2..cd20349883 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/EntityProperty.java @@ -14,12 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.admin.service.transaction; +package org.apache.nifi.admin.service.entity; /** - * + * Abstraction for stored properties of persistent Entities */ -public interface TransactionBuilder { - - Transaction start() throws TransactionException; +public interface EntityProperty { + /** + * Get property name for an element of an Entity + * + * @return Property name + */ + String getProperty(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/EntityType.java similarity index 60% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/EntityType.java index 141aa84882..ec94087f97 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/EntityType.java @@ -14,23 +14,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.admin.service.action; - -import org.apache.nifi.admin.dao.DAOFactory; +package org.apache.nifi.admin.service.entity; /** - * Defines the administration action. Actions are provided a DAO factory and - * authority provider to perform a require action. - * - * @param type + * Enumeration of Persistent Entity Types */ -public interface AdministrationAction { +public enum EntityType { + ACTION("Action"), - /** - * Performs an action using the specified DAOFactory and AuthorityProvider. - * - * @param daoFactory factory - * @return action result - */ - T execute(DAOFactory daoFactory); + CONFIGURE_DETAILS("ConfigureDetails"), + + CONNECT_DETAILS("ConnectDetails"), + + EXTENSION_DETAILS("ExtensionDetails"), + + MOVE_DETAILS("MoveDetails"), + + PURGE_DETAILS("PurgeDetails"); + + private final String entityType; + + EntityType(final String entityType) { + this.entityType = entityType; + } + + public String getEntityType() { + return entityType; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ExtensionDetailsEntity.java similarity index 66% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ExtensionDetailsEntity.java index 8fe1f8e9ed..ce7798aabb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/ExtensionDetailsEntity.java @@ -14,22 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.admin.dao; +package org.apache.nifi.admin.service.entity; /** - * Represents any error that might occur while administering NiFi accounts. + * Enumeration of Extension Details properties stored as Entity objects */ -public class DataAccessException extends RuntimeException { +public enum ExtensionDetailsEntity implements EntityProperty { + EXTENSION_TYPE("extensionType"); - public DataAccessException(Throwable cause) { - super(cause); + private final String property; + + ExtensionDetailsEntity(final String property) { + this.property = property; } - public DataAccessException(String message, Throwable cause) { - super(message, cause); - } - - public DataAccessException(String message) { - super(message); + @Override + public String getProperty() { + return property; } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/MoveDetailsEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/MoveDetailsEntity.java new file mode 100644 index 0000000000..2abdd9f0a1 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/MoveDetailsEntity.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service.entity; + +/** + * Enumeration of Action Move Details properties stored as Entity objects + */ +public enum MoveDetailsEntity implements EntityProperty { + ACTION("action"), + + GROUP("group"), + + GROUP_ID("groupId"), + + PREVIOUS_GROUP("previousGroup"), + + PREVIOUS_GROUP_ID("previousGroupId"); + + private final String property; + + MoveDetailsEntity(final String property) { + this.property = property; + } + + @Override + public String getProperty() { + return property; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/PurgeDetailsEntity.java similarity index 66% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/PurgeDetailsEntity.java index f6ebd151c3..7d20e7c517 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/entity/PurgeDetailsEntity.java @@ -14,26 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.admin.dao.impl; - -import org.apache.nifi.admin.dao.ActionDAO; -import org.apache.nifi.admin.dao.DAOFactory; - -import java.sql.Connection; +package org.apache.nifi.admin.service.entity; /** - * + * Enumeration of Action Purge Details properties stored as Entity objects */ -public class DAOFactoryImpl implements DAOFactory { +public enum PurgeDetailsEntity implements EntityProperty { + ACTION("action"), - private final Connection connection; + END_DATE("endDate"); - public DAOFactoryImpl(Connection connection) { - this.connection = connection; + private final String property; + + PurgeDetailsEntity(final String property) { + this.property = property; } @Override - public ActionDAO getActionDAO() { - return new StandardActionDAO(connection); + public String getProperty() { + return property; } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java deleted file mode 100644 index ba918cf71f..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java +++ /dev/null @@ -1,255 +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.admin.service.impl; - -import org.apache.nifi.action.Action; -import org.apache.nifi.admin.dao.DataAccessException; -import org.apache.nifi.admin.service.AdministrationException; -import org.apache.nifi.admin.service.AuditService; -import org.apache.nifi.admin.service.action.AddActionsAction; -import org.apache.nifi.admin.service.action.DeletePreviousValues; -import org.apache.nifi.admin.service.action.GetActionAction; -import org.apache.nifi.admin.service.action.GetActionsAction; -import org.apache.nifi.admin.service.action.GetPreviousValues; -import org.apache.nifi.admin.service.action.PurgeActionsAction; -import org.apache.nifi.admin.service.transaction.Transaction; -import org.apache.nifi.admin.service.transaction.TransactionBuilder; -import org.apache.nifi.admin.service.transaction.TransactionException; -import org.apache.nifi.history.History; -import org.apache.nifi.history.HistoryQuery; -import org.apache.nifi.history.PreviousValue; - -import java.io.IOException; -import java.util.Collection; -import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -/** - * - */ -public class StandardAuditService implements AuditService { - - private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); - private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock(); - - private TransactionBuilder transactionBuilder; - - @Override - public void addActions(Collection actions) { - Transaction transaction = null; - - writeLock.lock(); - try { - // start the transaction - transaction = transactionBuilder.start(); - - // seed the accounts - AddActionsAction addActions = new AddActionsAction(actions); - transaction.execute(addActions); - - // commit the transaction - transaction.commit(); - } catch (TransactionException | DataAccessException te) { - rollback(transaction); - throw new AdministrationException(te); - } catch (Throwable t) { - rollback(transaction); - throw t; - } finally { - closeQuietly(transaction); - writeLock.unlock(); - } - } - - @Override - public Map> getPreviousValues(String componentId) { - Transaction transaction = null; - Map> previousValues = null; - - readLock.lock(); - try { - // start the transaction - transaction = transactionBuilder.start(); - - // seed the accounts - GetPreviousValues getActions = new GetPreviousValues(componentId); - previousValues = transaction.execute(getActions); - - // commit the transaction - transaction.commit(); - } catch (TransactionException | DataAccessException te) { - rollback(transaction); - throw new AdministrationException(te); - } catch (Throwable t) { - rollback(transaction); - throw t; - } finally { - closeQuietly(transaction); - readLock.unlock(); - } - - return previousValues; - } - - @Override - public void deletePreviousValues(String propertyName, String componentId) { - Transaction transaction = null; - - readLock.lock(); - try { - // start the transaction - transaction = transactionBuilder.start(); - - // seed the accounts - DeletePreviousValues deleteAction = new DeletePreviousValues(propertyName, componentId); - transaction.execute(deleteAction); - - // commit the transaction - transaction.commit(); - } catch (TransactionException | DataAccessException te) { - rollback(transaction); - throw new AdministrationException(te); - } catch (Throwable t) { - rollback(transaction); - throw t; - } finally { - closeQuietly(transaction); - readLock.unlock(); - } - } - - @Override - public History getActions(HistoryQuery query) { - Transaction transaction = null; - History history = null; - - readLock.lock(); - try { - // start the transaction - transaction = transactionBuilder.start(); - - // seed the accounts - GetActionsAction getActions = new GetActionsAction(query); - history = transaction.execute(getActions); - - // commit the transaction - transaction.commit(); - } catch (TransactionException | DataAccessException te) { - rollback(transaction); - throw new AdministrationException(te); - } catch (Throwable t) { - rollback(transaction); - throw t; - } finally { - closeQuietly(transaction); - readLock.unlock(); - } - - return history; - } - - @Override - public History getActions(int firstActionId, int maxActions) { - final HistoryQuery query = new HistoryQuery(); - query.setOffset(firstActionId); - query.setCount(maxActions); - query.setSortOrder("asc"); - query.setSortColumn("timestamp"); - - return getActions(query); - } - - @Override - public Action getAction(Integer actionId) { - Transaction transaction = null; - Action action = null; - - readLock.lock(); - try { - // start the transaction - transaction = transactionBuilder.start(); - - // seed the accounts - GetActionAction getAction = new GetActionAction(actionId); - action = transaction.execute(getAction); - - // commit the transaction - transaction.commit(); - } catch (TransactionException | DataAccessException te) { - rollback(transaction); - throw new AdministrationException(te); - } catch (Throwable t) { - rollback(transaction); - throw t; - } finally { - closeQuietly(transaction); - readLock.unlock(); - } - - return action; - } - - @Override - public void purgeActions(Date end, Action purgeAction) { - Transaction transaction = null; - - writeLock.lock(); - try { - // start the transaction - transaction = transactionBuilder.start(); - - // purge the action database - PurgeActionsAction purgeActions = new PurgeActionsAction(end, purgeAction); - transaction.execute(purgeActions); - - // commit the transaction - transaction.commit(); - } catch (TransactionException | DataAccessException te) { - rollback(transaction); - throw new AdministrationException(te); - } catch (Throwable t) { - rollback(transaction); - throw t; - } finally { - closeQuietly(transaction); - writeLock.unlock(); - } - } - - private void rollback(Transaction transaction) { - if (transaction != null) { - transaction.rollback(); - } - } - - private void closeQuietly(final Transaction transaction) { - if (transaction != null) { - try { - transaction.close(); - } catch (final IOException ioe) { - } - } - } - - public void setTransactionBuilder(TransactionBuilder transactionBuilder) { - this.transactionBuilder = transactionBuilder; - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java deleted file mode 100644 index 4792581dc2..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java +++ /dev/null @@ -1,49 +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.admin.service.transaction; - -import java.io.Closeable; -import org.apache.nifi.admin.service.action.AdministrationAction; - -/** - * Defines a transaction. - */ -public interface Transaction extends Closeable { - - /** - * Executes the specified action within the current transaction. - * - * @param type of action to execute - * @param action action to execute - * @return executed action - * @throws IllegalStateException - if there is no current transaction - */ - T execute(AdministrationAction action); - - /** - * Commits the current transaction. - * - * @throws TransactionException - if the transaction is unable to be - * committed - */ - void commit() throws TransactionException; - - /** - * Rolls back the current transaction. - */ - void rollback(); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java deleted file mode 100644 index 1390768745..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java +++ /dev/null @@ -1,91 +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.admin.service.transaction.impl; - -import org.apache.nifi.admin.RepositoryUtils; -import org.apache.nifi.admin.dao.DAOFactory; -import org.apache.nifi.admin.dao.impl.DAOFactoryImpl; -import org.apache.nifi.admin.service.action.AdministrationAction; -import org.apache.nifi.admin.service.transaction.Transaction; -import org.apache.nifi.admin.service.transaction.TransactionException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.SQLException; - -/** - * Transaction implementation that uses the specified SQL Connection and - * AuthorityProvider. - */ -public class StandardTransaction implements Transaction { - - private static final Logger logger = LoggerFactory.getLogger(StandardTransaction.class); - - private Connection connection; - - public StandardTransaction(Connection connection) { - this.connection = connection; - } - - @Override - public T execute(AdministrationAction action) { - // ensure the transaction has been started - if (connection == null) { - throw new IllegalStateException("This transaction is not active."); - } - - // create a dao factory - DAOFactory daoFactory = new DAOFactoryImpl(connection); - - // execute the specified action - return action.execute(daoFactory); - } - - @Override - public void commit() throws TransactionException { - // ensure there is an active transaction - if (connection == null) { - throw new IllegalStateException("No active transaction."); - } - - try { - // commit the transaction - connection.commit(); - } catch (SQLException sqle) { - throw new TransactionException(sqle.getMessage()); - } - } - - @Override - public void rollback() { - // ensure there is an active transaction - if (connection != null) { - // rollback the transaction - RepositoryUtils.rollback(connection, logger); - } - } - - @Override - public void close() throws IOException { - if (connection != null) { - RepositoryUtils.closeQuietly(connection); - connection = null; - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java deleted file mode 100644 index e4b12180c2..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.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.admin.service.transaction.impl; - -import java.sql.Connection; -import java.sql.SQLException; -import java.sql.SQLFeatureNotSupportedException; -import javax.sql.DataSource; -import org.apache.nifi.admin.service.transaction.Transaction; -import org.apache.nifi.admin.service.transaction.TransactionBuilder; -import org.apache.nifi.admin.service.transaction.TransactionException; - -/** - * - */ -public class StandardTransactionBuilder implements TransactionBuilder { - - private DataSource dataSource; - - @Override - public Transaction start() throws TransactionException { - try { - // get a new connection - Connection connection = dataSource.getConnection(); - final boolean isAutoCommit = connection.getAutoCommit(); - if (isAutoCommit) { - try { - connection.setAutoCommit(false); - } catch (SQLFeatureNotSupportedException sfnse) { - throw new TransactionException("setAutoCommit(false) not supported by this driver"); - } - } - - // create a new transaction - return new StandardTransaction(connection); - } catch (SQLException sqle) { - throw new TransactionException(sqle.getMessage()); - } - } - - /* setters */ - public void setDataSource(DataSource dataSource) { - this.dataSource = dataSource; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml deleted file mode 100644 index f24ce960f3..0000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - - - - - - - - - - - - - - - - - diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/EntityStoreAuditServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/EntityStoreAuditServiceTest.java new file mode 100644 index 0000000000..9f48811d20 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/test/java/org/apache/nifi/admin/service/EntityStoreAuditServiceTest.java @@ -0,0 +1,369 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.FlowChangeAction; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.FlowChangeConfigureDetails; +import org.apache.nifi.action.details.FlowChangePurgeDetails; +import org.apache.nifi.action.details.PurgeDetails; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.PreviousValue; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledOnOs; +import org.junit.jupiter.api.condition.OS; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class EntityStoreAuditServiceTest { + + private static final Date ACTION_TIMESTAMP = new Date(86400); + + private static final Date SECOND_ACTION_TIMESTAMP = new Date(97500); + + private static final Date PURGE_END_DATE = new Date(43200); + + private static final String USER_IDENTITY = "admin"; + + private static final String SOURCE_ID = "00000000-0000-0000-0000-000000000000"; + + private static final String SECOND_SOURCE_ID = "01010101-0000-0000-0000-000000000000"; + + private static final String SOURCE_NAME = "GenerateFlowFile"; + + private static final Component SOURCE_TYPE = Component.Processor; + + private static final Operation OPERATION = Operation.Add; + + private static final int ACTION_ID = 0; + + private static final int PURGE_ACTIONS = 1; + + private static final String FIRST_PROPERTY_NAME = "FirstProperty"; + + private static final String SECOND_PROPERTY_NAME = "SecondProperty"; + + private static final String FIRST_VALUE = "FirstValue"; + + private static final String SECOND_VALUE = "SecondValue"; + + private static final String DATABASE_FILE_EXTENSION = ".xd"; + + @TempDir + File directory; + + private EntityStoreAuditService service; + + @BeforeEach + void setService() { + service = new EntityStoreAuditService(directory); + } + + @AfterEach + void closeService() throws IOException { + service.close(); + } + + @DisabledOnOs(value = OS.WINDOWS, disabledReason = "Moving the database lock file on Windows causes exceptions") + @Test + void testCorruptedFilesHandling() throws IOException { + service.close(); + + // Write invalid string + try (Stream files = Files.list(directory.toPath()).filter(file -> file.toString().endsWith(DATABASE_FILE_EXTENSION))) { + final Optional databaseFileFound = files.findFirst(); + assertTrue(databaseFileFound.isPresent()); + + final Path databaseFile = databaseFileFound.get(); + Files.writeString(databaseFile, SOURCE_ID); + } + + // Create Service with corrupted directory + service = new EntityStoreAuditService(directory); + final Action action = newAction(); + final Collection actions = Collections.singletonList(action); + service.addActions(actions); + } + + @Test + void testGetActionNotFound() { + final Action action = service.getAction(ACTION_ID); + + assertNull(action); + } + + @Test + void testAddActionsGetAction() { + final Action action = newAction(); + final Collection actions = Collections.singletonList(action); + + service.addActions(actions); + + final Action actionFound = service.getAction(ACTION_ID); + assertEquals(ACTION_ID, actionFound.getId()); + assertActionFound(actionFound); + } + + @Test + void testAddActionsGetActions() { + final Action firstAction = newAction(); + final Action secondAction = newAction(); + final Collection actions = Arrays.asList(firstAction, secondAction); + + service.addActions(actions); + + final History actionsHistory = service.getActions(ACTION_ID, actions.size()); + + assertNotNull(actionsHistory); + assertEquals(actions.size(), actionsHistory.getTotal()); + assertNotNull(actionsHistory.getLastRefreshed()); + + final Collection actionsFound = actionsHistory.getActions(); + assertNotNull(actionsFound); + for (final Action actionFound : actionsFound) { + assertActionFound(actionFound); + } + } + + @Test + void testAddActionsGetActionsQueryUnfiltered() { + final Action firstAction = newAction(); + final Action secondAction = newAction(); + final Collection actions = Arrays.asList(firstAction, secondAction); + + service.addActions(actions); + + final HistoryQuery historyQuery = new HistoryQuery(); + final History actionsHistory = service.getActions(historyQuery); + + assertNotNull(actionsHistory); + assertEquals(actions.size(), actionsHistory.getTotal()); + assertNotNull(actionsHistory.getLastRefreshed()); + + final Collection actionsFound = actionsHistory.getActions(); + assertNotNull(actionsFound); + for (final Action actionFound : actionsFound) { + assertActionFound(actionFound); + } + } + + @Test + void testAddActionsGetActionsQuerySourceNotFound() { + final Action firstAction = newAction(); + final Action secondAction = newAction(); + final Collection actions = Arrays.asList(firstAction, secondAction); + + service.addActions(actions); + + final HistoryQuery historyQuery = new HistoryQuery(); + historyQuery.setSourceId(SECOND_SOURCE_ID); + + final History actionsHistory = service.getActions(historyQuery); + + assertNotNull(actionsHistory); + assertEquals(0, actionsHistory.getTotal()); + assertNotNull(actionsHistory.getLastRefreshed()); + + final Collection actionsFound = actionsHistory.getActions(); + assertNotNull(actionsFound); + assertTrue(actionsFound.isEmpty()); + } + + @Test + void testAddActionsGetActionsQueryStartDateFiltered() { + final FlowChangeAction firstAction = newAction(); + final FlowChangeAction secondAction = newAction(); + secondAction.setTimestamp(SECOND_ACTION_TIMESTAMP); + final Collection actions = Arrays.asList(firstAction, secondAction); + + service.addActions(actions); + + final HistoryQuery historyQuery = new HistoryQuery(); + historyQuery.setStartDate(SECOND_ACTION_TIMESTAMP); + final History actionsHistory = service.getActions(historyQuery); + + assertNotNull(actionsHistory); + assertEquals(actionsHistory.getTotal(), 1); + assertNotNull(actionsHistory.getLastRefreshed()); + + final Collection actionsFound = actionsHistory.getActions(); + assertNotNull(actionsFound); + + final Iterator actionsFiltered = actionsFound.iterator(); + assertTrue(actionsFiltered.hasNext()); + + final Action firstActionFound = actionsFiltered.next(); + assertEquals(SECOND_ACTION_TIMESTAMP, firstActionFound.getTimestamp()); + + assertFalse(actionsFiltered.hasNext()); + } + + @Test + void testAddActionsPurgeActionsGetAction() { + final Action action = newAction(); + final Collection actions = Collections.singletonList(action); + + service.addActions(actions); + + final FlowChangeAction purgeAction = newAction(); + purgeAction.setOperation(Operation.Purge); + + final FlowChangePurgeDetails purgeDetails = new FlowChangePurgeDetails(); + purgeDetails.setEndDate(PURGE_END_DATE); + purgeAction.setActionDetails(purgeDetails); + + service.purgeActions(new Date(), purgeAction); + + final History history = service.getActions(ACTION_ID, PURGE_ACTIONS); + assertNotNull(history); + assertEquals(PURGE_ACTIONS, history.getTotal()); + + final Iterator actionsFound = history.getActions().iterator(); + assertTrue(actionsFound.hasNext()); + + final Action actionFound = actionsFound.next(); + assertEquals(Operation.Purge, actionFound.getOperation()); + + final ActionDetails actionDetails = actionFound.getActionDetails(); + assertInstanceOf(PurgeDetails.class, actionDetails); + + final PurgeDetails purgeDetailsFound = (PurgeDetails) actionDetails; + assertEquals(PURGE_END_DATE, purgeDetailsFound.getEndDate()); + } + + @Test + void testAddActionsDeletePreviousValuesGetActions() { + final FlowChangeAction firstAction = newAction(); + firstAction.setOperation(Operation.Configure); + final FlowChangeConfigureDetails firstConfigureDetails = new FlowChangeConfigureDetails(); + firstConfigureDetails.setName(FIRST_PROPERTY_NAME); + firstConfigureDetails.setValue(FIRST_VALUE); + firstAction.setActionDetails(firstConfigureDetails); + + final FlowChangeAction secondAction = newAction(); + secondAction.setOperation(Operation.Configure); + final FlowChangeConfigureDetails secondConfigureDetails = new FlowChangeConfigureDetails(); + secondConfigureDetails.setName(SECOND_PROPERTY_NAME); + secondConfigureDetails.setValue(SECOND_VALUE); + secondAction.setActionDetails(secondConfigureDetails); + + final Collection actions = Arrays.asList(firstAction, secondAction); + service.addActions(actions); + + service.deletePreviousValues(SECOND_PROPERTY_NAME, SOURCE_ID); + + final History actionsHistory = service.getActions(ACTION_ID, Integer.MAX_VALUE); + assertNotNull(actionsHistory); + assertEquals(actions.size(), actionsHistory.getTotal()); + + final Iterator actionsFound = actionsHistory.getActions().iterator(); + assertTrue(actionsFound.hasNext()); + + final Action firstActionFound = actionsFound.next(); + assertNotNull(firstActionFound.getActionDetails()); + + assertTrue(actionsFound.hasNext()); + final Action secondActionFound = actionsFound.next(); + assertNull(secondActionFound.getActionDetails()); + } + + @Test + void testAddActionsGetPreviousValues() { + final FlowChangeAction firstAction = newAction(); + firstAction.setOperation(Operation.Configure); + final FlowChangeConfigureDetails firstConfigureDetails = new FlowChangeConfigureDetails(); + firstConfigureDetails.setName(FIRST_PROPERTY_NAME); + firstConfigureDetails.setValue(FIRST_VALUE); + firstAction.setActionDetails(firstConfigureDetails); + + final FlowChangeAction secondAction = newAction(); + secondAction.setOperation(Operation.Configure); + final FlowChangeConfigureDetails secondConfigureDetails = new FlowChangeConfigureDetails(); + secondConfigureDetails.setName(SECOND_PROPERTY_NAME); + secondConfigureDetails.setValue(SECOND_VALUE); + secondAction.setActionDetails(secondConfigureDetails); + + final Collection actions = Arrays.asList(firstAction, secondAction); + service.addActions(actions); + + final Map> previousValues = service.getPreviousValues(SOURCE_ID); + assertNotNull(previousValues); + assertFalse(previousValues.isEmpty()); + + final List firstPreviousValues = previousValues.get(FIRST_PROPERTY_NAME); + assertNotNull(firstPreviousValues); + final PreviousValue firstPreviousValue = firstPreviousValues.get(0); + assertNotNull(firstPreviousValue); + assertEquals(FIRST_VALUE, firstPreviousValue.getPreviousValue()); + assertNotNull(firstPreviousValue.getTimestamp()); + assertEquals(USER_IDENTITY, firstPreviousValue.getUserIdentity()); + + final List secondPreviousValues = previousValues.get(SECOND_PROPERTY_NAME); + assertNotNull(secondPreviousValues); + final PreviousValue secondPreviousValue = secondPreviousValues.get(0); + assertNotNull(secondPreviousValue); + assertEquals(SECOND_VALUE, secondPreviousValue.getPreviousValue()); + assertNotNull(secondPreviousValue.getTimestamp()); + assertEquals(USER_IDENTITY, secondPreviousValue.getUserIdentity()); + } + + private FlowChangeAction newAction() { + final FlowChangeAction action = new FlowChangeAction(); + action.setTimestamp(ACTION_TIMESTAMP); + action.setSourceId(SOURCE_ID); + action.setSourceName(SOURCE_NAME); + action.setSourceType(SOURCE_TYPE); + action.setUserIdentity(USER_IDENTITY); + action.setOperation(OPERATION); + return action; + } + + private void assertActionFound(final Action actionFound) { + assertNotNull(actionFound); + assertEquals(ACTION_TIMESTAMP, actionFound.getTimestamp()); + assertEquals(SOURCE_ID, actionFound.getSourceId()); + assertEquals(SOURCE_NAME, actionFound.getSourceName()); + assertEquals(USER_IDENTITY, actionFound.getUserIdentity()); + assertEquals(OPERATION, actionFound.getOperation()); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/nifi.properties index dc5764fee0..b335ff843e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/nifi.properties @@ -41,9 +41,8 @@ nifi.state.management.embedded.zookeeper.start=false # Properties file that provides the ZooKeeper properties to use if is set to true nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.VolatileFlowFileRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/conf/nifi.properties index c3971d803b..ccd38320f4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/conf/nifi.properties @@ -28,9 +28,8 @@ nifi.ui.autorefresh.interval=30 sec nifi.nar.library.directory=./target/lib nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml index c39a8f984e..61fd72429d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml @@ -151,10 +151,6 @@ nifi-stateless-engine 2.0.0-SNAPSHOT - - com.h2database - h2 - org.bouncycastle bcprov-jdk18on @@ -179,6 +175,14 @@ commons-io commons-io + + org.apache.commons + commons-collections4 + + + org.aspectj + aspectjweaver + org.apache.commons commons-math3 diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties index 040ea85acc..db203c315b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/conf/nifi.properties @@ -28,9 +28,8 @@ nifi.ui.autorefresh.interval=30 sec nifi.nar.library.directory=./target/lib nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties index 484325558a..6088f2efd4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flowcontrollertest.nifi.properties @@ -31,9 +31,8 @@ nifi.nar.working.directory=./target/flowcontrollertest/work/nar/ nifi.state.management.configuration.file=src/test/resources/state-management.xml nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./target/flowcontrollertest/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/flowcontrollertest/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties index 2ed45db046..b6fa1b5fb2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi-with-remote.properties @@ -28,9 +28,8 @@ nifi.ui.autorefresh.interval=30 sec nifi.nar.library.directory=./target/lib nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowserializertest.nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowserializertest.nifi.properties index 1a5515bd99..fd3ed76748 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowserializertest.nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowserializertest.nifi.properties @@ -31,9 +31,8 @@ nifi.nar.working.directory=./target/standardflowserializertest/work/nar/ nifi.state.management.configuration.file=src/test/resources/state-management.xml nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./target/standardflowserializertest/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/standardflowserializertest/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowsynchronizerspec.nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowsynchronizerspec.nifi.properties index 303c2e9cca..9d8e3c968a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowsynchronizerspec.nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardflowsynchronizerspec.nifi.properties @@ -31,9 +31,8 @@ nifi.nar.working.directory=./target/standardflowsynchronizerspec/work/nar/ nifi.state.management.configuration.file=src/test/resources/state-management.xml nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./target/standardflowsynchronizerspec/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/standardflowsynchronizerspec/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardprocessschedulertest.nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardprocessschedulertest.nifi.properties index 5bb0baf925..647ffe5348 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardprocessschedulertest.nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/standardprocessschedulertest.nifi.properties @@ -31,9 +31,8 @@ nifi.nar.working.directory=./target/standardprocessschedulertest/work/nar/ nifi.state.management.configuration.file=src/test/resources/state-management.xml nifi.state.management.provider.local=local-provider -# H2 Settings +# Database Settings nifi.database.directory=./target/standardprocessschedulertest/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/standardprocessschedulertest/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_with_native_lib.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_with_native_lib.properties index fa3be8a5ac..13e124e521 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_with_native_lib.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_with_native_lib.properties @@ -31,9 +31,8 @@ nifi.nar.library.autoload.directory=./target/nars_with_native_lib nifi.nar.working.directory=./target/work/nar/ nifi.documentation.working.directory=./target/work/docs/components -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_without_native_lib.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_without_native_lib.properties index b018334b53..3f8f8653a6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_without_native_lib.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.nar_without_native_lib.properties @@ -31,9 +31,8 @@ nifi.nar.library.autoload.directory=./target/nars_without_native_lib nifi.nar.working.directory=./target/work/nar/ nifi.documentation.working.directory=./target/work/docs/components -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties index 534e47b729..e9219bdcf4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties @@ -31,9 +31,8 @@ nifi.nar.library.autoload.directory=./target/extensions nifi.nar.working.directory=./target/work/nar/ nifi.documentation.working.directory=./target/work/docs/components -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties index 4183ce1b9f..724639cf9d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties @@ -31,9 +31,8 @@ nifi.nar.library.directory.alt=./target/NarUnpacker/lib2/ nifi.nar.working.directory=./target/work/nar/ nifi.documentation.working.directory=./target/work/docs/components -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties index 2a63a02187..475301e95e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties @@ -48,9 +48,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessAuditService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessAuditService.java new file mode 100644 index 0000000000..a1bb41c486 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessAuditService.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.headless; + +import org.apache.nifi.action.Action; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.PreviousValue; + +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; + +/** + * Headless Audit Service implementation does not persist actions + */ +public class HeadlessAuditService implements AuditService { + @Override + public void addActions(Collection actions) { + + } + + @Override + public Map> getPreviousValues(String componentId) { + return null; + } + + @Override + public void deletePreviousValues(String propertyName, String componentId) { + + } + + @Override + public History getActions(HistoryQuery actionQuery) { + return null; + } + + @Override + public History getActions(int firstActionId, int maxActions) { + return null; + } + + @Override + public Action getAction(Integer actionId) { + return null; + } + + @Override + public void purgeActions(Date end, Action purgeAction) { + + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java index 592b708472..c0da6303f8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java @@ -18,7 +18,6 @@ package org.apache.nifi.headless; import org.apache.nifi.NiFiServer; import org.apache.nifi.admin.service.AuditService; -import org.apache.nifi.admin.service.impl.StandardAuditService; import org.apache.nifi.authorization.AuthorizationRequest; import org.apache.nifi.authorization.AuthorizationResult; import org.apache.nifi.authorization.Authorizer; @@ -101,7 +100,7 @@ public class HeadlessNiFiServer implements NiFiServer { logger.info("Loading Flow..."); FlowFileEventRepository flowFileEventRepository = new RingBufferEventRepository(5); - AuditService auditService = new StandardAuditService(); + AuditService auditService = new HeadlessAuditService(); Authorizer authorizer = new Authorizer() { @Override public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties index f11eaba2c3..d1c7cc5bee 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties @@ -48,9 +48,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml index b6be9e8261..ec52c4e71e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml @@ -84,8 +84,6 @@ NIFI_PBKDF2_AES_GCM_256 - ;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE - 9990 true diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml index 3b7c7d1043..b13218d62f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml @@ -192,6 +192,9 @@ + + + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties index aea5a693dc..e53b291dcc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties @@ -66,9 +66,8 @@ nifi.state.management.embedded.zookeeper.start=${nifi.state.management.embedded. # Properties file that provides the ZooKeeper properties to use if is set to true nifi.state.management.embedded.zookeeper.properties=${nifi.state.management.embedded.zookeeper.properties} -# H2 Settings +# Database Settings nifi.database.directory=${nifi.database.directory} -nifi.h2.url.append=${nifi.h2.url.append} # Repository Encryption properties override individual repository implementation properties nifi.repository.encryption.protocol.version= diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/encrypted.nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/encrypted.nifi.properties index b554ebb245..359752d005 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/encrypted.nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/encrypted.nifi.properties @@ -49,9 +49,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties=./target/conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/nifi.properties index 374f5b6e97..bf7efdb117 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/resources/NiFiProperties/conf/nifi.properties @@ -48,9 +48,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties=./target/conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties index 403deae865..bb8b90d840 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/nifi.properties @@ -33,10 +33,6 @@ nifi.flowcontroller.graceful.shutdown.seconds=10 nifi.nar.library.directory=./lib nifi.nar.working.directory=./work/nar/ nifi.flowservice.writedelay.seconds=2 -nifi.h2.repository.maxmemoryrows=100000 -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE -nifi.h2.max.connections=20 -nifi.h2.login.timeout=500 #For testing purposes. Default value should actually be empty! nifi.remote.input.socket.port=5000 nifi.remote.input.secure=true diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiConfiguration.java index c2f5f42ae7..2bed7b2bb4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiConfiguration.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiConfiguration.java @@ -16,11 +16,18 @@ */ package org.apache.nifi.web; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.admin.service.EntityStoreAuditService; +import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.security.configuration.WebSecurityConfiguration; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; import org.springframework.context.annotation.Import; import org.springframework.context.annotation.ImportResource; +import java.io.File; + /** * Web Application Spring Configuration */ @@ -29,7 +36,6 @@ import org.springframework.context.annotation.ImportResource; WebSecurityConfiguration.class }) @ImportResource({"classpath:nifi-context.xml", - "classpath:nifi-administration-context.xml", "classpath:nifi-authorizer-context.xml", "classpath:nifi-cluster-manager-context.xml", "classpath:nifi-cluster-protocol-context.xml", @@ -40,4 +46,16 @@ public class NiFiWebApiConfiguration { super(); } + /** + * Audit Service implementation from nifi-administration + * + * @param properties NiFi Properties + * @return Audit Service implementation using Persistent Entity Store + */ + @Autowired + @Bean + public AuditService auditService(final NiFiProperties properties) { + final File databaseDirectory = properties.getDatabaseRepositoryPath().toFile(); + return new EntityStoreAuditService(databaseDirectory); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-anonymous-allowed.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-anonymous-allowed.properties index 3a76886725..1a9ca2c4d0 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-anonymous-allowed.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-anonymous-allowed.properties @@ -34,9 +34,8 @@ nifi.state.management.embedded.zookeeper.max.instances=3 nifi.state.management.provider.local=local-provider nifi.state.management.provider.cluster= -# H2 Settings +# Database Settings nifi.database.directory=target/test-classes/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-flow.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-flow.properties index 5f2dce9716..124aa3c78e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-flow.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-flow.properties @@ -34,9 +34,8 @@ nifi.state.management.embedded.zookeeper.max.instances=3 nifi.state.management.provider.local=local-provider nifi.state.management.provider.cluster= -# H2 Settings +# Database Settings nifi.database.directory=target/test-classes/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-mapped-identities.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-mapped-identities.properties index 99cc893bf8..2d9ba13471 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-mapped-identities.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi-mapped-identities.properties @@ -40,9 +40,8 @@ nifi.state.management.embedded.zookeeper.max.instances=3 nifi.state.management.provider.local=local-provider nifi.state.management.provider.cluster= -# H2 Settings +# Database Settings nifi.database.directory=target/test-classes/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties index 82acb5d0fa..854f384f6d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties @@ -34,9 +34,8 @@ nifi.state.management.embedded.zookeeper.max.instances=3 nifi.state.management.provider.local=local-provider nifi.state.management.provider.cluster= -# H2 Settings +# Database Settings nifi.database.directory=target/test-classes/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties index 0aa55881de..ade760a297 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties @@ -45,9 +45,8 @@ nifi.state.management.embedded.zookeeper.start=${nifi.state.management.embedded. nifi.state.management.embedded.zookeeper.properties=${nifi.state.management.embedded.zookeeper.properties} -# H2 Settings +# Database Settings nifi.database.directory=${nifi.database.directory} -nifi.h2.url.append=${nifi.h2.url.append} # FlowFile Repository nifi.flowfile.repository.implementation=${nifi.flowfile.repository.implementation} diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml index 0a649f4739..944230e264 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml @@ -260,11 +260,6 @@ quartz 2.3.2 - - com.h2database - h2 - ${h2.version} - diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/nifi.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/nifi.properties index f0702251f1..2ef362d5ef 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/nifi.properties +++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/nifi.properties @@ -64,9 +64,8 @@ nifi.state.management.embedded.zookeeper.start=true nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/nifi.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/nifi.properties index 0f98fe124b..e5e505b2af 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/nifi.properties +++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/nifi.properties @@ -64,9 +64,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties= -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/nifi.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/nifi.properties index bc50975d30..6ce80350c4 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/nifi.properties +++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/nifi.properties @@ -64,9 +64,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/pythonic/nifi.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/pythonic/nifi.properties index 108df0a627..d454c4b3b5 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/pythonic/nifi.properties +++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/pythonic/nifi.properties @@ -68,9 +68,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_default.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_default.properties index 8922c4e89c..b0c6f0a85d 100644 --- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_default.properties +++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_default.properties @@ -29,9 +29,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes.properties index 4b419b427a..e7eb189122 100644 --- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes.properties +++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes.properties @@ -29,9 +29,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_128.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_128.properties index 777f0d03f8..f93e7ed825 100644 --- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_128.properties +++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_128.properties @@ -29,9 +29,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password.properties index 62df5a541a..27b28c2bfc 100644 --- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password.properties +++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password.properties @@ -29,9 +29,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password_128.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password_128.properties index 43002ef121..d3c2e321a6 100644 --- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password_128.properties +++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_protected_aes_password_128.properties @@ -30,9 +30,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected.properties index 962b9b2b7d..8951cad4aa 100644 --- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected.properties +++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected.properties @@ -29,9 +29,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected_and_empty_protection_schemes.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected_and_empty_protection_schemes.properties index 2d1a0e66a5..d416c46bcf 100644 --- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected_and_empty_protection_schemes.properties +++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi_with_sensitive_properties_unprotected_and_empty_protection_schemes.properties @@ -30,9 +30,8 @@ nifi.nar.library.directory=./target/resources/NiFiProperties/lib/ nifi.nar.library.directory.alt=./target/resources/NiFiProperties/lib2/ nifi.nar.working.directory=./target/work/nar/ -# H2 Settings +# Database Settings nifi.database.directory=./target/database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.directory=./target/test-repo diff --git a/nifi-toolkit/nifi-toolkit-tls/src/test/resources/localhost/nifi.properties b/nifi-toolkit/nifi-toolkit-tls/src/test/resources/localhost/nifi.properties index 9d5f6b483c..1d85410299 100644 --- a/nifi-toolkit/nifi-toolkit-tls/src/test/resources/localhost/nifi.properties +++ b/nifi-toolkit/nifi-toolkit-tls/src/test/resources/localhost/nifi.properties @@ -48,9 +48,8 @@ nifi.state.management.embedded.zookeeper.start=false nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties -# H2 Settings +# Database Settings nifi.database.directory=./database_repository -nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE # FlowFile Repository nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository