From 7d7401add4699cd7d543ce18d11d7c9d6ab6e20c Mon Sep 17 00:00:00 2001
From: joewitt
Date: Mon, 15 Aug 2016 20:18:47 -0700
Subject: [PATCH 1/4] NIFI-2574 Changed NiFiProperties to avoid static
initializer and updated all references to it.
---
...ontrollerServiceInitializationContext.java | 3 +-
.../apache/nifi/kerberos/KerberosContext.java | 51 ++++
.../ProcessorInitializationContext.java | 11 +-
.../ReportingInitializationContext.java | 3 +-
nifi-commons/nifi-hadoop-utils/pom.xml | 4 -
.../nifi/hadoop/KerberosProperties.java | 36 +--
.../nifi/hadoop/TestKerberosProperties.java | 12 +-
.../org/apache/nifi/util/NiFiProperties.java | 236 +++++++++-------
.../apache/nifi/util/NiFiPropertiesTest.java | 52 +---
...ontrollerServiceInitializationContext.java | 16 ++
.../MockProcessorInitializationContext.java | 16 ++
.../MockReportingInitializationContext.java | 16 ++
...ontrollerServiceInitializationContext.java | 15 +
.../MockProcessorInitializationContext.java | 16 ++
.../MockReportingInitializationContext.java | 16 ++
.../nifi/documentation/DocGeneratorTest.java | 35 ++-
.../authorization/FileAuthorizerTest.java | 2 +-
.../util/IdentityMappingUtil.java | 2 +-
...dardClusterCoordinationProtocolSender.java | 8 +-
.../heartbeat/AbstractHeartbeatMonitor.java | 39 ++-
.../ClusterProtocolHeartbeatMonitor.java | 40 ++-
.../http/StandardHttpResponseMerger.java | 34 ++-
.../StatusHistoryEndpointMerger.java | 18 +-
.../ThreadPoolRequestReplicator.java | 12 +-
.../node/CuratorNodeProtocolSender.java | 18 +-
.../node/NodeClusterCoordinator.java | 100 +++----
...hreadPoolRequestReplicatorFactoryBean.java | 18 +-
.../TestAbstractHeartbeatMonitor.java | 14 +-
.../StandardHttpResponseMergerSpec.groovy | 7 +-
.../StatusHistoryEndpointMergerSpec.groovy | 5 +-
.../TestThreadPoolRequestReplicator.java | 32 +--
.../node/TestNodeClusterCoordinator.java | 24 +-
.../nifi/cluster/integration/Cluster.java | 10 +-
.../apache/nifi/cluster/integration/Node.java | 27 +-
.../nifi/remote/protocol/ServerProtocol.java | 12 +-
.../nifi/connectable/StandardConnection.java | 19 +-
.../controller/FileSystemSwapManager.java | 53 ++--
.../nifi/controller/FlowController.java | 80 +++---
.../nifi/controller/StandardFlowService.java | 103 +++----
.../controller/StandardFlowSynchronizer.java | 61 ++--
.../controller/StandardProcessorNode.java | 10 +-
.../cluster/ClusterProtocolHeartbeater.java | 20 +-
.../cluster/ZooKeeperClientConfig.java | 15 +-
.../CuratorLeaderElectionManager.java | 31 +--
...tandardReportingInitializationContext.java | 25 +-
.../repository/FileSystemRepository.java | 86 +++---
.../repository/VolatileContentRepository.java | 44 ++-
.../WriteAheadFlowFileRepository.java | 50 +++-
.../scheduling/StandardProcessScheduler.java | 29 +-
.../TimerDrivenSchedulingAgent.java | 11 +-
...ontrollerServiceInitializationContext.java | 24 +-
.../StandardControllerServiceProvider.java | 7 +-
.../VolatileComponentStatusRepository.java | 11 +-
.../apache/nifi/encrypt/StringEncryptor.java | 21 +-
.../StandardXMLFlowConfigurationDAO.java | 8 +-
...tandardProcessorInitializationContext.java | 24 +-
.../remote/StandardRemoteProcessGroup.java | 78 +++---
.../StandardFlowSynchronizerSpec.groovy | 8 +-
.../controller/StandardFlowServiceTest.java | 2 +-
.../controller/TestFileSystemSwapManager.java | 10 +-
.../nifi/controller/TestFlowController.java | 26 +-
.../controller/TestStandardProcessorNode.java | 5 +-
.../repository/TestFileSystemRepository.java | 101 +++----
.../TestStandardProcessSession.java | 260 +++++++++---------
.../TestVolatileContentRepository.java | 33 +--
.../TestWriteAheadFlowFileRepository.java | 30 +-
.../scheduling/TestProcessorLifecycle.java | 73 +++--
.../TestStandardProcessScheduler.java | 64 +++--
...StandardControllerServiceProviderTest.java | 14 +-
...TestStandardControllerServiceProvider.java | 40 +--
.../zookeeper/TestZooKeeperStateProvider.java | 17 +-
.../TestContinuallyRunProcessorTask.java | 5 +-
.../nifi/nar/NarThreadContextClassLoader.java | 31 ++-
.../org/apache/nifi/nar/NarUnpackerTest.java | 58 ++--
.../src/main/java/org/apache/nifi/NiFi.java | 2 +-
.../security/util/SslServerSocketFactory.java | 4 +-
.../security/util/SslSocketFactory.java | 4 +-
.../nifi/remote/HttpRemoteSiteListener.java | 30 +-
.../nifi/remote/RemoteResourceFactory.java | 4 +-
.../nifi/remote/SocketRemoteSiteListener.java | 19 +-
.../nifi/remote/StandardRemoteGroupPort.java | 52 ++--
.../AbstractFlowFileServerProtocol.java | 9 +-
.../remote/protocol/FlowFileTransaction.java | 2 +-
.../remote/protocol/HandshakeProperties.java | 1 -
.../StandardHttpFlowFileServerProtocol.java | 19 +-
.../socket/SocketFlowFileServerProtocol.java | 25 +-
.../remote/TestHttpRemoteSiteListener.java | 11 +-
.../remote/TestStandardRemoteGroupPort.java | 10 +-
.../io/socket/TestSocketChannelStreams.java | 2 +-
.../io/socket/ssl/TestSSLSocketChannel.java | 12 +-
.../http/TestHttpFlowFileServerProtocol.java | 10 +-
.../nifi/web/server/JettyServerTest.java | 33 +--
.../nifi/web/api/DataTransferResource.java | 130 +++++----
.../nifi/web/api/SiteToSiteResource.java | 43 ++-
.../accesscontrol/AccessControlHelper.java | 10 +-
.../accesscontrol/ITAccessTokenEndpoint.java | 5 +-
.../nifi/integration/util/NiFiTestServer.java | 19 +-
.../web/api/TestDataTransferResource.java | 18 +-
.../nifi/web/api/TestSiteToSiteResource.java | 4 +-
.../OcspCertificateValidatorGroovyTest.groovy | 12 +-
.../NiFiAuthenticationProviderTest.java | 2 +-
.../hadoop/AbstractHadoopProcessor.java | 9 +-
.../processors/hadoop/AbstractHadoopTest.java | 5 +-
.../nifi/processors/hadoop/GetHDFSTest.java | 2 +-
.../nifi/processors/hadoop/PutHDFSTest.java | 2 +-
.../hadoop/TestCreateHadoopSequenceFile.java | 2 +-
.../nifi/processors/hadoop/TestFetchHDFS.java | 2 +-
.../nifi/processors/hadoop/TestListHDFS.java | 2 +-
.../hadoop/inotify/TestGetHDFSEvents.java | 2 +-
.../nifi/dbcp/hive/HiveConnectionPool.java | 29 +-
.../processors/hive/PutHiveStreaming.java | 6 +-
.../processors/hive/TestPutHiveStreaming.java | 16 +-
.../PersistentProvenanceRepository.java | 218 +++++++++------
.../VolatileProvenanceRepository.java | 45 +--
.../TestVolatileProvenanceRepository.java | 7 +-
.../script/InvokeScriptedProcessor.java | 115 +++++---
.../nifi/controller/MonitorDiskUsage.java | 62 ++---
.../nifi/controller/MonitorMemoryTest.java | 25 +-
.../nifi/hbase/HBase_1_1_2_ClientService.java | 10 +-
.../hbase/TestHBase_1_1_2_ClientService.java | 9 +-
120 files changed, 2034 insertions(+), 1573 deletions(-)
create mode 100644 nifi-api/src/main/java/org/apache/nifi/kerberos/KerberosContext.java
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java
index 3486621c7633..f6ac9e768c9a 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java
@@ -17,9 +17,10 @@
package org.apache.nifi.controller;
import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.kerberos.KerberosContext;
import org.apache.nifi.logging.ComponentLog;
-public interface ControllerServiceInitializationContext {
+public interface ControllerServiceInitializationContext extends KerberosContext {
/**
* @return the identifier associated with the {@link ControllerService} with
diff --git a/nifi-api/src/main/java/org/apache/nifi/kerberos/KerberosContext.java b/nifi-api/src/main/java/org/apache/nifi/kerberos/KerberosContext.java
new file mode 100644
index 000000000000..8f7ba192d90c
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/kerberos/KerberosContext.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.kerberos;
+
+import java.io.File;
+
+public interface KerberosContext {
+
+ /**
+ * The Kerberos service principal used by NiFi to communicate with the KDC
+ * in order to obtain tickets on behalf of NiFi. Typically of the form
+ * NIFI/fully.qualified.domain@REALM.
+ *
+ * @return the principal, or null if this NiFi instance is not configured
+ * with a NiFi Kerberos service principal
+ */
+ public String getKerberosServicePrincipal();
+
+ /**
+ * The File instance for the Kerberos service keytab. The service principal
+ * and service keytab will be used to communicate with the KDC to obtain
+ * tickets on behalf of NiFi.
+ *
+ * @return the File instance of the service keytab, or null if this NiFi
+ * instance is not configured with a NiFi Kerberos service keytab
+ */
+ public File getKerberosServiceKeytab();
+
+ /**
+ * The Kerberos configuration file (typically krb5.conf) that will be used
+ * by this JVM during all Kerberos operations.
+ *
+ * @return the File instance for the Kerberos configuration file, or null if
+ * this NiFi instance is not configured with a Kerberos configuration file
+ */
+ public File getKerberosConfigurationFile();
+}
diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessorInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessorInitializationContext.java
index 726b3fa6b321..df1819376a85 100644
--- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessorInitializationContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessorInitializationContext.java
@@ -18,16 +18,17 @@
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.kerberos.KerberosContext;
import org.apache.nifi.logging.ComponentLog;
/**
*
* The ProcessorInitializationContext provides
- * {@link org.apache.nifi.processor.Processor Processor}s access to objects that may be of
- * use throughout the life of the Processor.
+ * {@link org.apache.nifi.processor.Processor Processor}s access to objects that
+ * may be of use throughout the life of the Processor.
*
*/
-public interface ProcessorInitializationContext {
+public interface ProcessorInitializationContext extends KerberosContext {
/**
* @return the unique identifier for this processor
@@ -47,7 +48,9 @@ public interface ProcessorInitializationContext {
ControllerServiceLookup getControllerServiceLookup();
/**
- * @return the {@link NodeTypeProvider} which can be used to detect the node type of this NiFi instance.
+ * @return the {@link NodeTypeProvider} which can be used to detect the node
+ * type of this NiFi instance.
*/
NodeTypeProvider getNodeTypeProvider();
+
}
diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java
index d014b26023a3..df64e03493a7 100644
--- a/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java
@@ -19,6 +19,7 @@
import java.util.concurrent.TimeUnit;
import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.kerberos.KerberosContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.scheduling.SchedulingStrategy;
@@ -26,7 +27,7 @@
* A ReportingConfiguration provides configuration information to a
* ReportingTask at the time of initialization
*/
-public interface ReportingInitializationContext {
+public interface ReportingInitializationContext extends KerberosContext {
/**
* @return the identifier for this ReportingTask
diff --git a/nifi-commons/nifi-hadoop-utils/pom.xml b/nifi-commons/nifi-hadoop-utils/pom.xml
index 5f184008a006..8de849e99bf5 100644
--- a/nifi-commons/nifi-hadoop-utils/pom.xml
+++ b/nifi-commons/nifi-hadoop-utils/pom.xml
@@ -34,10 +34,6 @@
org.apache.nifi
nifi-processor-utils
-
- org.apache.nifi
- nifi-properties
-
org.apache.hadoop
hadoop-common
diff --git a/nifi-commons/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java b/nifi-commons/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java
index 5e8fb7dd889f..c7743f4ef299 100644
--- a/nifi-commons/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java
+++ b/nifi-commons/nifi-hadoop-utils/src/main/java/org/apache/nifi/hadoop/KerberosProperties.java
@@ -23,20 +23,20 @@
import org.apache.nifi.components.Validator;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.StringUtils;
import java.io.File;
import java.util.ArrayList;
import java.util.List;
/**
- * All processors and controller services that need properties for Kerberos Principal and Keytab
- * should obtain them through this class by calling:
+ * All processors and controller services that need properties for Kerberos
+ * Principal and Keytab should obtain them through this class by calling:
*
- * KerberosProperties props = KerberosProperties.create(NiFiProperties.getInstance())
+ * KerberosProperties props =
+ * KerberosProperties.create(NiFiProperties.getInstance())
*
- * The properties can be accessed from the resulting KerberosProperties instance.
+ * The properties can be accessed from the resulting KerberosProperties
+ * instance.
*/
public class KerberosProperties {
@@ -45,7 +45,14 @@ public class KerberosProperties {
private final PropertyDescriptor kerberosPrincipal;
private final PropertyDescriptor kerberosKeytab;
- private KerberosProperties(final File kerberosConfigFile) {
+ /**
+ * Instantiate a KerberosProperties object but keep in mind it is
+ * effectively a singleton because the krb5.conf file needs to be set as a
+ * system property which this constructor will take care of.
+ *
+ * @param kerberosConfigFile file of krb5.conf
+ */
+ public KerberosProperties(final File kerberosConfigFile) {
this.kerberosConfigFile = kerberosConfigFile;
if (this.kerberosConfigFile != null) {
@@ -91,13 +98,6 @@ public ValidationResult validate(String subject, String input, ValidationContext
.build();
}
- public static KerberosProperties create(final NiFiProperties niFiProperties) {
- if (niFiProperties == null) {
- throw new IllegalArgumentException("NiFiProperties can not be null");
- }
- return new KerberosProperties(niFiProperties.getKerberosConfigurationFile());
- }
-
public File getKerberosConfigFile() {
return kerberosConfigFile;
}
@@ -120,7 +120,8 @@ public static List validatePrincipalAndKeytab(final String sub
// if security is enabled then the keytab and principal are required
final boolean isSecurityEnabled = SecurityUtil.isSecurityEnabled(config);
- if (isSecurityEnabled && StringUtils.isBlank(principal)) {
+ final boolean blankPrincipal = (principal == null || principal.isEmpty());
+ if (isSecurityEnabled && blankPrincipal) {
results.add(new ValidationResult.Builder()
.valid(false)
.subject(subject)
@@ -128,7 +129,8 @@ public static List validatePrincipalAndKeytab(final String sub
.build());
}
- if (isSecurityEnabled && StringUtils.isBlank(keytab)) {
+ final boolean blankKeytab = (keytab == null || keytab.isEmpty());
+ if (isSecurityEnabled && blankKeytab) {
results.add(new ValidationResult.Builder()
.valid(false)
.subject(subject)
@@ -136,7 +138,7 @@ public static List validatePrincipalAndKeytab(final String sub
.build());
}
- if (!isSecurityEnabled && (!StringUtils.isBlank(principal) || !StringUtils.isBlank(keytab))) {
+ if (!isSecurityEnabled && (!blankPrincipal || !blankKeytab)) {
logger.warn("Configuration does not have security enabled, Keytab and Principal will be ignored");
}
diff --git a/nifi-commons/nifi-hadoop-utils/src/test/java/org/apache/nifi/hadoop/TestKerberosProperties.java b/nifi-commons/nifi-hadoop-utils/src/test/java/org/apache/nifi/hadoop/TestKerberosProperties.java
index 131fe656e903..8cd1ea1d3561 100644
--- a/nifi-commons/nifi-hadoop-utils/src/test/java/org/apache/nifi/hadoop/TestKerberosProperties.java
+++ b/nifi-commons/nifi-hadoop-utils/src/test/java/org/apache/nifi/hadoop/TestKerberosProperties.java
@@ -19,7 +19,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.util.NiFiProperties;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
@@ -27,17 +26,13 @@
import java.io.File;
import java.util.List;
-import static org.mockito.Mockito.when;
-
public class TestKerberosProperties {
@Test
public void testWithKerberosConfigFile() {
final File file = new File("src/test/resources/krb5.conf");
- final NiFiProperties niFiProperties = Mockito.mock(NiFiProperties.class);
- when(niFiProperties.getKerberosConfigurationFile()).thenReturn(file);
- final KerberosProperties kerberosProperties = KerberosProperties.create(niFiProperties);
+ final KerberosProperties kerberosProperties = new KerberosProperties(file);
Assert.assertNotNull(kerberosProperties);
Assert.assertNotNull(kerberosProperties.getKerberosConfigFile());
@@ -51,10 +46,9 @@ public void testWithKerberosConfigFile() {
@Test
public void testWithoutKerberosConfigFile() {
- final NiFiProperties niFiProperties = Mockito.mock(NiFiProperties.class);
- when(niFiProperties.getKerberosConfigurationFile()).thenReturn(null);
+ final File file = new File("src/test/resources/krb5.conf");
- final KerberosProperties kerberosProperties = KerberosProperties.create(niFiProperties);
+ final KerberosProperties kerberosProperties = new KerberosProperties(null);
Assert.assertNotNull(kerberosProperties);
Assert.assertNull(kerberosProperties.getKerberosConfigFile());
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 bbb3998a626e..d381f740a883 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
@@ -16,9 +16,6 @@
*/
package org.apache.nifi.util;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.io.BufferedInputStream;
import java.io.File;
import java.io.FileInputStream;
@@ -29,17 +26,22 @@
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
-
-public class NiFiProperties extends Properties {
-
- private static final long serialVersionUID = 2119177359005492702L;
-
- private static final Logger LOG = LoggerFactory.getLogger(NiFiProperties.class);
- private static NiFiProperties instance = null;
+import java.util.Set;
+
+/**
+ * The NiFiProperties class holds all properties which are needed for various
+ * values to be available at runtime. It is strongly tied to the startup
+ * properties needed and is often refer to as the 'nifi.properties' file. The
+ * properties contains keys and values. Great care should be taken in leveraging
+ * this class or passing it along. It's use should be refactored and minimized
+ * over time.
+ */
+public abstract class NiFiProperties {
// core properties
public static final String PROPERTIES_FILE_PATH = "nifi.properties.file.path";
@@ -174,7 +176,6 @@ public class NiFiProperties extends Properties {
public static final String ZOOKEEPER_SESSION_TIMEOUT = "nifi.zookeeper.session.timeout";
public static final String ZOOKEEPER_ROOT_NODE = "nifi.zookeeper.root.node";
-
// kerberos properties
public static final String KERBEROS_KRB5_FILE = "nifi.kerberos.krb5.file";
public static final String KERBEROS_SERVICE_PRINCIPAL = "nifi.kerberos.service.principal";
@@ -243,68 +244,20 @@ public class NiFiProperties extends Properties {
// Kerberos defaults
public static final String DEFAULT_KERBEROS_AUTHENTICATION_EXPIRATION = "12 hours";
- private NiFiProperties() {
- super();
- }
-
- public NiFiProperties copy() {
- final NiFiProperties copy = new NiFiProperties();
- copy.putAll(this);
- return copy;
- }
+ /**
+ * Retrieves the property value for the given property key
+ *
+ * @param key the key of property value to lookup.
+ * @return value of property at given key or null if not found
+ */
+ public abstract String getProperty(String key);
/**
- * Factory method to create an instance of the {@link NiFiProperties}. This
- * method employs a standard singleton pattern by caching the instance if it
- * was already obtained
+ * Retrieves all known property keys.
*
- * @return instance of {@link NiFiProperties}
+ * @return all known property keys.
*/
- public static synchronized NiFiProperties getInstance() {
- // NOTE: unit tests can set instance to null (with reflection) to effectively create a new singleton.
- // changing the below as a check for whether the instance was initialized will break those
- // unit tests.
- if (null == instance) {
- final NiFiProperties suspectInstance = new NiFiProperties();
- final String nfPropertiesFilePath = System
- .getProperty(NiFiProperties.PROPERTIES_FILE_PATH);
- if (null == nfPropertiesFilePath || nfPropertiesFilePath.trim().length() == 0) {
- throw new RuntimeException("Requires a system property called \'"
- + NiFiProperties.PROPERTIES_FILE_PATH
- + "\' and this is not set or has no value");
- }
- final File propertiesFile = new File(nfPropertiesFilePath);
- if (!propertiesFile.exists()) {
- throw new RuntimeException("Properties file doesn't exist \'"
- + propertiesFile.getAbsolutePath() + "\'");
- }
- if (!propertiesFile.canRead()) {
- throw new RuntimeException("Properties file exists but cannot be read \'"
- + propertiesFile.getAbsolutePath() + "\'");
- }
- InputStream inStream = null;
- try {
- inStream = new BufferedInputStream(new FileInputStream(propertiesFile));
- suspectInstance.load(inStream);
- } catch (final Exception ex) {
- LOG.error("Cannot load properties file due to " + ex.getLocalizedMessage());
- throw new RuntimeException("Cannot load properties file due to "
- + ex.getLocalizedMessage(), ex);
- } finally {
- if (null != inStream) {
- try {
- inStream.close();
- } catch (final Exception ex) {
- /**
- * do nothing *
- */
- }
- }
- }
- instance = suspectInstance;
- }
- return instance;
- }
+ public abstract Set getPropertyKeys();
// getters for core properties //
public File getFlowConfigurationFile() {
@@ -395,7 +348,8 @@ public String getAdministrativeYieldDuration() {
}
/**
- * The host name that will be given out to clients to connect to the Remote Input Port.
+ * The host name that will be given out to clients to connect to the Remote
+ * Input Port.
*
* @return the remote input host name or null if not configured
*/
@@ -443,7 +397,9 @@ public Boolean isSiteToSiteHttpEnabled() {
/**
* The HTTP or HTTPS Web API port for a Remote Input Port.
- * @return the remote input port for HTTP(S) communication, or null if HTTP(S) Site-to-Site is not enabled
+ *
+ * @return the remote input port for HTTP(S) communication, or null if
+ * HTTP(S) Site-to-Site is not enabled
*/
public Integer getRemoteInputHttpPort() {
if (!isSiteToSiteHttpEnabled()) {
@@ -479,7 +435,8 @@ public String getFlowServiceWriteDelay() {
}
/**
- * Returns whether the processors should be started automatically when the application loads.
+ * Returns whether the processors should be started automatically when the
+ * application loads.
*
* @return Whether to auto start the processors or not
*/
@@ -490,7 +447,8 @@ public boolean getAutoResumeState() {
}
/**
- * Returns the number of partitions that should be used for the FlowFile Repository
+ * Returns the number of partitions that should be used for the FlowFile
+ * Repository
*
* @return the number of partitions
*/
@@ -501,7 +459,8 @@ public int getFlowFileRepositoryPartitions() {
}
/**
- * Returns the number of milliseconds between FlowFileRepository checkpointing
+ * Returns the number of milliseconds between FlowFileRepository
+ * checkpointing
*
* @return the number of milliseconds between checkpoint events
*/
@@ -608,7 +567,7 @@ public List getNarLibraryDirectories() {
List narLibraryPaths = new ArrayList<>();
// go through each property
- for (String propertyName : stringPropertyNames()) {
+ for (String propertyName : getPropertyKeys()) {
// determine if the property is a nar library path
if (StringUtils.startsWith(propertyName, NAR_LIBRARY_DIRECTORY_PREFIX)
|| NAR_LIBRARY_DIRECTORY.equals(propertyName)) {
@@ -684,7 +643,6 @@ public File getPersistentStateDirectory() {
return file;
}
-
// getters for cluster node properties //
public boolean isNode() {
return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE));
@@ -719,7 +677,6 @@ public int getClusterNodeProtocolThreads() {
}
}
-
public boolean isClustered() {
return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE));
}
@@ -779,7 +736,8 @@ public String getKerberosAuthenticationExpiration() {
}
/**
- * Returns true if the Kerberos service principal and keytab location properties are populated.
+ * Returns true if the Kerberos service principal and keytab location
+ * properties are populated.
*
* @return true if Kerberos service support is enabled
*/
@@ -788,12 +746,14 @@ public boolean isKerberosServiceSupportEnabled() {
}
/**
- * Returns true if client certificates are required for REST API. Determined if the following conditions are all true:
+ * Returns true if client certificates are required for REST API. Determined
+ * if the following conditions are all true:
*
- * - login identity provider is not populated
- * - Kerberos service support is not enabled
+ * - login identity provider is not populated - Kerberos service support is
+ * not enabled
*
- * @return true if client certificates are required for access to the REST API
+ * @return true if client certificates are required for access to the REST
+ * API
*/
public boolean isClientAuthRequiredForRestApi() {
return StringUtils.isBlank(getProperty(NiFiProperties.SECURITY_USER_LOGIN_IDENTITY_PROVIDER)) && !isKerberosServiceSupportEnabled();
@@ -839,7 +799,8 @@ public InetSocketAddress getNodeApiAddress() {
}
/**
- * Returns the database repository path. It simply returns the value configured. No directories will be created as a result of this operation.
+ * Returns the database repository path. It simply returns the value
+ * configured. No directories will be created as a result of this operation.
*
* @return database repository path
* @throws InvalidPathException If the configured path is invalid
@@ -849,7 +810,8 @@ public Path getDatabaseRepositoryPath() {
}
/**
- * Returns the flow file repository path. It simply returns the value configured. No directories will be created as a result of this operation.
+ * Returns the flow file repository path. It simply returns the value
+ * configured. No directories will be created as a result of this operation.
*
* @return database repository path
* @throws InvalidPathException If the configured path is invalid
@@ -859,8 +821,10 @@ public Path getFlowFileRepositoryPath() {
}
/**
- * Returns the content repository paths. This method returns a mapping of file repository name to file repository paths. It simply returns the values configured. No directories will be created as
- * a result of this operation.
+ * Returns the content repository paths. This method returns a mapping of
+ * file repository name to file repository paths. It simply returns the
+ * values configured. No directories will be created as a result of this
+ * operation.
*
* @return file repositories paths
* @throws InvalidPathException If any of the configured paths are invalid
@@ -869,7 +833,7 @@ public Map getContentRepositoryPaths() {
final Map contentRepositoryPaths = new HashMap<>();
// go through each property
- for (String propertyName : stringPropertyNames()) {
+ for (String propertyName : getPropertyKeys()) {
// determine if the property is a file repository path
if (StringUtils.startsWith(propertyName, REPOSITORY_CONTENT_PREFIX)) {
// get the repository key
@@ -884,8 +848,10 @@ public Map getContentRepositoryPaths() {
}
/**
- * Returns the provenance repository paths. This method returns a mapping of file repository name to file repository paths. It simply returns the values configured. No directories will be created
- * as a result of this operation.
+ * Returns the provenance repository paths. This method returns a mapping of
+ * file repository name to file repository paths. It simply returns the
+ * values configured. No directories will be created as a result of this
+ * operation.
*
* @return the name and paths of all provenance repository locations
*/
@@ -893,7 +859,7 @@ public Map getProvenanceRepositoryPaths() {
final Map provenanceRepositoryPaths = new HashMap<>();
// go through each property
- for (String propertyName : stringPropertyNames()) {
+ for (String propertyName : getPropertyKeys()) {
// determine if the property is a file repository path
if (StringUtils.startsWith(propertyName, PROVENANCE_REPO_DIRECTORY_PREFIX)) {
// get the repository key
@@ -919,17 +885,9 @@ public String getMaxAppendableClaimSize() {
return getProperty(MAX_APPENDABLE_CLAIM_SIZE);
}
- @Override
public String getProperty(final String key, final String defaultValue) {
- final String value = super.getProperty(key, defaultValue);
- if (value == null) {
- return null;
- }
-
- if (value.trim().isEmpty()) {
- return defaultValue;
- }
- return value;
+ final String value = getProperty(key);
+ return (value == null || value.trim().isEmpty()) ? defaultValue : value;
}
public String getBoredYieldDuration() {
@@ -973,7 +931,7 @@ public String getFlowConfigurationArchiveMaxStorage() {
return getProperty(FLOW_CONFIGURATION_ARCHIVE_MAX_STORAGE, DEFAULT_FLOW_CONFIGURATION_ARCHIVE_MAX_STORAGE);
}
- public String getVariableRegistryProperties(){
+ public String getVariableRegistryProperties() {
return getProperty(VARIABLE_REGISTRY_PROPERTIES);
}
@@ -981,18 +939,88 @@ public Path[] getVariableRegistryPropertiesPaths() {
final List vrPropertiesPaths = new ArrayList<>();
final String vrPropertiesFiles = getVariableRegistryProperties();
- if(!StringUtils.isEmpty(vrPropertiesFiles)) {
+ if (!StringUtils.isEmpty(vrPropertiesFiles)) {
final List vrPropertiesFileList = Arrays.asList(vrPropertiesFiles.split(","));
- for(String propertiesFile : vrPropertiesFileList){
+ for (String propertiesFile : vrPropertiesFileList) {
vrPropertiesPaths.add(Paths.get(propertiesFile));
}
- return vrPropertiesPaths.toArray( new Path[vrPropertiesPaths.size()]);
+ return vrPropertiesPaths.toArray(new Path[vrPropertiesPaths.size()]);
} else {
return new Path[]{};
}
}
+ public int size() {
+ return getPropertyKeys().size();
+ }
+
+ /**
+ * Creates an instance of NiFiProperties. This should likely not be called
+ * by any classes outside of the NiFi framework but can be useful by the
+ * framework for default property loading behavior or helpful in tests
+ * needing to create specific instances of NiFiProperties. If properties
+ * file specified cannot be found/read a runtime exception will be thrown.
+ * If one is not specified no properties will be loaded by default.
+ *
+ * @param propertiesFilePath if provided properties will be loaded from
+ * given file; else will be loaded from System property. Can be null.
+ * @param additionalProperties allows overriding of properties with the
+ * supplied values. these will be applied after loading from any properties
+ * file. Can be null or empty.
+ * @return NiFiProperties
+ */
+ public static NiFiProperties createBasicNiFiProperties(final String propertiesFilePath, final Map additionalProperties) {
+ final Map addProps = (additionalProperties == null) ? Collections.EMPTY_MAP : additionalProperties;
+ final Properties properties = new Properties();
+ final String nfPropertiesFilePath = (propertiesFilePath == null)
+ ? System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH)
+ : propertiesFilePath;
+ if (nfPropertiesFilePath != null) {
+ final File propertiesFile = new File(nfPropertiesFilePath.trim());
+ if (!propertiesFile.exists()) {
+ throw new RuntimeException("Properties file doesn't exist \'"
+ + propertiesFile.getAbsolutePath() + "\'");
+ }
+ if (!propertiesFile.canRead()) {
+ throw new RuntimeException("Properties file exists but cannot be read \'"
+ + propertiesFile.getAbsolutePath() + "\'");
+ }
+ InputStream inStream = null;
+ try {
+ inStream = new BufferedInputStream(new FileInputStream(propertiesFile));
+ properties.load(inStream);
+ } catch (final Exception ex) {
+ throw new RuntimeException("Cannot load properties file due to "
+ + ex.getLocalizedMessage(), ex);
+ } finally {
+ if (null != inStream) {
+ try {
+ inStream.close();
+ } catch (final Exception ex) {
+ /**
+ * do nothing *
+ */
+ }
+ }
+ }
+ }
+ addProps.entrySet().stream().forEach((entry) -> {
+ properties.setProperty(entry.getKey(), entry.getValue());
+ });
+ return new NiFiProperties() {
+ @Override
+ public String getProperty(String key) {
+ return properties.getProperty(key);
+ }
+
+ @Override
+ public Set getPropertyKeys() {
+ return properties.stringPropertyNames();
+ }
+ };
+ }
+
}
diff --git a/nifi-commons/nifi-properties/src/test/java/org/apache/nifi/util/NiFiPropertiesTest.java b/nifi-commons/nifi-properties/src/test/java/org/apache/nifi/util/NiFiPropertiesTest.java
index 96a618e10ebc..3547b92822d1 100644
--- a/nifi-commons/nifi-properties/src/test/java/org/apache/nifi/util/NiFiPropertiesTest.java
+++ b/nifi-commons/nifi-properties/src/test/java/org/apache/nifi/util/NiFiPropertiesTest.java
@@ -19,10 +19,7 @@
import org.junit.Assert;
import org.junit.Test;
-import java.io.BufferedInputStream;
import java.io.File;
-import java.io.FileInputStream;
-import java.io.InputStream;
import java.net.URISyntaxException;
import java.nio.file.Path;
import java.util.HashSet;
@@ -36,7 +33,7 @@ public class NiFiPropertiesTest {
@Test
public void testProperties() {
- NiFiProperties properties = loadSpecifiedProperties("/NiFiProperties/conf/nifi.properties");
+ NiFiProperties properties = loadNiFiProperties("/NiFiProperties/conf/nifi.properties");
assertEquals("UI Banner Text", properties.getBannerText());
@@ -58,7 +55,7 @@ public void testProperties() {
@Test
public void testMissingProperties() {
- NiFiProperties properties = loadSpecifiedProperties("/NiFiProperties/conf/nifi.missing.properties");
+ NiFiProperties properties = loadNiFiProperties("/NiFiProperties/conf/nifi.missing.properties");
List directories = properties.getNarLibraryDirectories();
@@ -72,7 +69,7 @@ public void testMissingProperties() {
@Test
public void testBlankProperties() {
- NiFiProperties properties = loadSpecifiedProperties("/NiFiProperties/conf/nifi.blank.properties");
+ NiFiProperties properties = loadNiFiProperties("/NiFiProperties/conf/nifi.blank.properties");
List directories = properties.getNarLibraryDirectories();
@@ -83,45 +80,14 @@ public void testBlankProperties() {
}
- private NiFiProperties loadSpecifiedProperties(String propertiesFile) {
-
- String filePath;
+ private NiFiProperties loadNiFiProperties(final String propsPath) {
+ String realPath = null;
try {
- filePath = NiFiPropertiesTest.class.getResource(propertiesFile).toURI().getPath();
- } catch (URISyntaxException ex) {
- throw new RuntimeException("Cannot load properties file due to "
- + ex.getLocalizedMessage(), ex);
- }
-
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, filePath);
-
- NiFiProperties properties = NiFiProperties.getInstance();
-
- // clear out existing properties
- for (String prop : properties.stringPropertyNames()) {
- properties.remove(prop);
+ realPath = NiFiPropertiesTest.class.getResource(propsPath).toURI().getPath();
+ } catch (final URISyntaxException ex) {
+ throw new RuntimeException(ex);
}
-
- InputStream inStream = null;
- try {
- inStream = new BufferedInputStream(new FileInputStream(filePath));
- properties.load(inStream);
- } catch (final Exception ex) {
- throw new RuntimeException("Cannot load properties file due to "
- + ex.getLocalizedMessage(), ex);
- } finally {
- if (null != inStream) {
- try {
- inStream.close();
- } catch (final Exception ex) {
- /**
- * do nothing *
- */
- }
- }
- }
-
- return properties;
+ return NiFiProperties.createBasicNiFiProperties(realPath, null);
}
}
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java
index f8f4f5995d40..ffa3546e897c 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.util;
+import java.io.File;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceInitializationContext;
@@ -68,4 +69,19 @@ public ComponentLog getLogger() {
public StateManager getStateManager() {
return stateManager;
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return null; //this needs to be wired in.
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return null; //this needs to be wired in.
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return null; //this needs to be wired in.
+ }
}
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java
index 132869c4094c..e44e7311deb6 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.util;
+import java.io.File;
import java.util.Set;
import java.util.UUID;
@@ -86,4 +87,19 @@ public boolean isControllerServiceEnabling(final String serviceIdentifier) {
public NodeTypeProvider getNodeTypeProvider() {
return context;
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return null; //this needs to be wired in.
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return null; //this needs to be wired in.
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return null; //this needs to be wired in.
+ }
}
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java
index 0aea00afadec..454b74293656 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.util;
+import java.io.File;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
@@ -86,4 +87,19 @@ public SchedulingStrategy getSchedulingStrategy() {
public ComponentLog getLogger() {
return logger;
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return null; //this needs to be wired in.
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return null; //this needs to be wired in.
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return null; //this needs to be wired in.
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java
index abbde61191c1..195934b4cd44 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.documentation.mock;
+import java.io.File;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerServiceInitializationContext;
import org.apache.nifi.controller.ControllerServiceLookup;
@@ -49,4 +50,18 @@ public StateManager getStateManager() {
return null;
}
+ @Override
+ public String getKerberosServicePrincipal() {
+ return null;
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return null;
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return null;
+ }
}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java
index 3935124be76d..d86ea4c9bd24 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.documentation.mock;
+import java.io.File;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.NodeTypeProvider;
import org.apache.nifi.logging.ComponentLog;
@@ -48,4 +49,19 @@ public ControllerServiceLookup getControllerServiceLookup() {
public NodeTypeProvider getNodeTypeProvider() {
return new MockNodeTypeProvider();
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return null;
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return null;
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return null;
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java
index ebf59d61dfe6..49d7933a5c64 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.documentation.mock;
+import java.io.File;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.controller.ControllerServiceLookup;
@@ -64,4 +65,19 @@ public SchedulingStrategy getSchedulingStrategy() {
public ComponentLog getLogger() {
return new MockComponentLogger();
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return null;
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return null;
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return null;
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
index afc3d50db3c3..b7c4db93c48f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
@@ -21,6 +21,8 @@
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
+import java.util.Properties;
+import java.util.Set;
import org.apache.commons.io.FileUtils;
import org.apache.nifi.nar.ExtensionManager;
@@ -38,8 +40,9 @@ public void testProcessorLoadsNarResources() throws IOException, ClassNotFoundEx
TemporaryFolder temporaryFolder = new TemporaryFolder();
temporaryFolder.create();
- NiFiProperties properties = loadSpecifiedProperties("/conf/nifi.properties");
- properties.setProperty(NiFiProperties.COMPONENT_DOCS_DIRECTORY, temporaryFolder.getRoot().getAbsolutePath());
+ NiFiProperties properties = loadSpecifiedProperties("/conf/nifi.properties",
+ NiFiProperties.COMPONENT_DOCS_DIRECTORY,
+ temporaryFolder.getRoot().getAbsolutePath());
NarUnpacker.unpackNars(properties);
@@ -60,22 +63,16 @@ public void testProcessorLoadsNarResources() throws IOException, ClassNotFoundEx
Assert.assertTrue(generatedHtml.contains("resources"));
}
- private NiFiProperties loadSpecifiedProperties(String propertiesFile) {
+ private NiFiProperties loadSpecifiedProperties(final String propertiesFile, final String key, final String value) {
String file = DocGeneratorTest.class.getResource(propertiesFile).getFile();
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, file);
- NiFiProperties properties = NiFiProperties.getInstance();
-
- // clear out existing properties
- for (String prop : properties.stringPropertyNames()) {
- properties.remove(prop);
- }
-
+ final Properties props = new Properties();
InputStream inStream = null;
try {
inStream = new BufferedInputStream(new FileInputStream(file));
- properties.load(inStream);
+ props.load(inStream);
} catch (final Exception ex) {
throw new RuntimeException("Cannot load properties file due to "
+ ex.getLocalizedMessage(), ex);
@@ -91,6 +88,20 @@ private NiFiProperties loadSpecifiedProperties(String propertiesFile) {
}
}
- return properties;
+ if (key != null && value != null) {
+ props.setProperty(key, value);
+ }
+
+ return new NiFiProperties() {
+ @Override
+ public String getProperty(String key) {
+ return props.getProperty(key);
+ }
+
+ @Override
+ public Set getPropertyKeys() {
+ return props.stringPropertyNames();
+ }
+ };
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
index 565e55a559eb..fcedfc82c428 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
@@ -1468,7 +1468,7 @@ private static boolean deleteFile(final File file) {
private NiFiProperties getNiFiProperties(final Properties properties) {
final NiFiProperties nifiProperties = Mockito.mock(NiFiProperties.class);
- when(nifiProperties.stringPropertyNames()).thenReturn(properties.stringPropertyNames());
+ when(nifiProperties.getPropertyKeys()).thenReturn(properties.stringPropertyNames());
when(nifiProperties.getProperty(anyString())).then(new Answer() {
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/util/IdentityMappingUtil.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/util/IdentityMappingUtil.java
index 2485db59f859..f7087ac28448 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/util/IdentityMappingUtil.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/util/IdentityMappingUtil.java
@@ -43,7 +43,7 @@ public static List getIdentityMappings(final NiFiProperties pro
final List mappings = new ArrayList<>();
// go through each property
- for (String propertyName : properties.stringPropertyNames()) {
+ for (String propertyName : properties.getPropertyKeys()) {
if (StringUtils.startsWith(propertyName, NiFiProperties.SECURITY_IDENTITY_MAPPING_PATTERN_PREFIX)) {
final String key = StringUtils.substringAfter(propertyName, NiFiProperties.SECURITY_IDENTITY_MAPPING_PATTERN_PREFIX);
final String identityPattern = properties.getProperty(propertyName);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java
index b9ff0829f0a6..167ddec93284 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java
@@ -48,7 +48,6 @@
import org.apache.nifi.io.socket.SocketUtils;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -63,6 +62,7 @@
*
*/
public class StandardClusterCoordinationProtocolSender implements ClusterCoordinationProtocolSender {
+
private static final Logger logger = LoggerFactory.getLogger(StandardClusterCoordinationProtocolSender.class);
private final ProtocolContext protocolContext;
@@ -70,10 +70,6 @@ public class StandardClusterCoordinationProtocolSender implements ClusterCoordin
private final int maxThreadsPerRequest;
private int handshakeTimeoutSeconds;
- public StandardClusterCoordinationProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) {
- this(socketConfiguration, protocolContext, NiFiProperties.getInstance().getClusterNodeProtocolThreads());
- }
-
public StandardClusterCoordinationProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext, final int maxThreadsPerRequest) {
if (socketConfiguration == null) {
throw new IllegalArgumentException("Socket configuration may not be null.");
@@ -90,7 +86,6 @@ public StandardClusterCoordinationProtocolSender(final SocketConfiguration socke
public void setBulletinRepository(final BulletinRepository bulletinRepository) {
}
-
/**
* Requests a node to reconnect to the cluster. The configured value for
* handshake timeout is applied to the socket before making the request.
@@ -158,7 +153,6 @@ public void disconnect(final DisconnectMessage msg) throws ProtocolException {
}
}
-
private void setConnectionHandshakeTimeoutOnSocket(final Socket socket) throws SocketException {
// update socket timeout, if handshake timeout was set; otherwise use socket's current timeout
if (handshakeTimeoutSeconds >= 0) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java
index c216ed32c29a..d2db59e31139 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.heartbeat;
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
@@ -31,7 +30,6 @@
import org.slf4j.LoggerFactory;
import java.util.Map;
-import java.util.Properties;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
@@ -45,11 +43,10 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
private volatile ScheduledFuture> future;
private volatile boolean stopped = true;
-
- public AbstractHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final Properties properties) {
+ public AbstractHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final NiFiProperties nifiProperties) {
this.clusterCoordinator = clusterCoordinator;
- final String heartbeatInterval = properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL,
- NiFiProperties.DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL);
+ final String heartbeatInterval = nifiProperties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL,
+ NiFiProperties.DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL);
this.heartbeatIntervalMillis = (int) FormatUtils.getTimeDuration(heartbeatInterval, TimeUnit.MILLISECONDS);
}
@@ -118,8 +115,8 @@ protected long getHeartbeatInterval(final TimeUnit timeUnit) {
}
/**
- * Fetches all of the latest heartbeats and updates the Cluster Coordinator as appropriate,
- * based on the heartbeats received.
+ * Fetches all of the latest heartbeats and updates the Cluster Coordinator
+ * as appropriate, based on the heartbeats received.
*
* Visible for testing.
*/
@@ -145,7 +142,7 @@ protected synchronized void monitorHeartbeats() {
processHeartbeat(heartbeat);
} catch (final Exception e) {
clusterCoordinator.reportEvent(null, Severity.ERROR,
- "Received heartbeat from " + heartbeat.getNodeIdentifier() + " but failed to process heartbeat due to " + e);
+ "Received heartbeat from " + heartbeat.getNodeIdentifier() + " but failed to process heartbeat due to " + e);
logger.error("Failed to process heartbeat from {} due to {}", heartbeat.getNodeIdentifier(), e.toString());
logger.error("", e);
}
@@ -162,7 +159,7 @@ protected synchronized void monitorHeartbeats() {
final long secondsSinceLastHeartbeat = TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - heartbeat.getTimestamp());
clusterCoordinator.disconnectionRequestedByNode(heartbeat.getNodeIdentifier(), DisconnectionCode.LACK_OF_HEARTBEAT,
- "Have not received a heartbeat from node in " + secondsSinceLastHeartbeat + " seconds");
+ "Have not received a heartbeat from node in " + secondsSinceLastHeartbeat + " seconds");
try {
removeHeartbeat(heartbeat.getNodeIdentifier());
@@ -201,8 +198,8 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) {
if (heartbeat.getConnectionStatus().getState() != NodeConnectionState.CONNECTED && connectionState == NodeConnectionState.CONNECTED) {
// Cluster Coordinator believes that node is connected, but node does not believe so.
clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Received heartbeat from node that thinks it is not yet part of the cluster,"
- + "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState()
- + "). Marking as Disconnected and requesting that Node reconnect to cluster");
+ + "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState()
+ + "). Marking as Disconnected and requesting that Node reconnect to cluster");
clusterCoordinator.requestNodeConnect(nodeId, null);
return;
}
@@ -220,7 +217,7 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) {
case NOT_YET_CONNECTED:
case STARTUP_FAILURE: {
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node previously "
- + "disconnected due to " + disconnectionCode + ". Issuing reconnection request.");
+ + "disconnected due to " + disconnectionCode + ". Issuing reconnection request.");
clusterCoordinator.requestNodeConnect(nodeId, null);
break;
@@ -260,23 +257,25 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) {
clusterCoordinator.updateNodeRoles(nodeId, heartbeat.getRoles());
}
-
/**
- * @return the most recent heartbeat information for each node in the cluster
+ * @return the most recent heartbeat information for each node in the
+ * cluster
*/
protected abstract Map getLatestHeartbeats();
/**
- * This method does nothing in the abstract class but is meant for subclasses to
- * override in order to provide functionality when the monitor is started.
+ * This method does nothing in the abstract class but is meant for
+ * subclasses to override in order to provide functionality when the monitor
+ * is started.
*/
protected void onStart() {
}
/**
- * This method does nothing in the abstract class but is meant for subclasses to
- * override in order to provide functionality when the monitor is stopped.
+ * This method does nothing in the abstract class but is meant for
+ * subclasses to override in order to provide functionality when the monitor
+ * is stopped.
*/
protected void onStop() {
}
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
index d2d81d1da46e..95b2045e4ec4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.heartbeat;
import java.nio.charset.StandardCharsets;
@@ -22,7 +21,6 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
-import java.util.Properties;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -54,10 +52,12 @@
import org.slf4j.LoggerFactory;
/**
- * Uses Apache ZooKeeper to advertise the address to send heartbeats to, and then relies on the NiFi Cluster
- * Protocol to receive heartbeat messages from nodes in the cluster.
+ * Uses Apache ZooKeeper to advertise the address to send heartbeats to, and
+ * then relies on the NiFi Cluster Protocol to receive heartbeat messages from
+ * nodes in the cluster.
*/
public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor implements HeartbeatMonitor, ProtocolHandler {
+
protected static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeatMonitor.class);
private static final String COORDINATOR_ZNODE_NAME = "coordinator";
@@ -81,30 +81,29 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
}
}
-
- public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final ProtocolListener protocolListener, final Properties properties) {
- super(clusterCoordinator, properties);
+ public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final ProtocolListener protocolListener, final NiFiProperties nifiProperties) {
+ super(clusterCoordinator, nifiProperties);
protocolListener.addHandler(this);
- this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties);
+ this.zkClientConfig = ZooKeeperClientConfig.createConfig(nifiProperties);
this.clusterNodesPath = zkClientConfig.resolvePath("cluster/nodes");
- String hostname = properties.getProperty(NiFiProperties.CLUSTER_NODE_ADDRESS);
+ String hostname = nifiProperties.getProperty(NiFiProperties.CLUSTER_NODE_ADDRESS);
if (hostname == null || hostname.trim().isEmpty()) {
hostname = "localhost";
}
- final String port = properties.getProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT);
+ final String port = nifiProperties.getProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT);
if (port == null || port.trim().isEmpty()) {
throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '"
- + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is not set");
+ + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is not set");
}
try {
Integer.parseInt(port);
} catch (final NumberFormatException nfe) {
throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '"
- + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number.");
+ + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number.");
}
heartbeatAddress = hostname + ":" + port;
@@ -114,12 +113,12 @@ public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinat
public void onStart() {
final RetryPolicy retryPolicy = new RetryForever(5000);
curatorClient = CuratorFrameworkFactory.builder()
- .connectString(zkClientConfig.getConnectString())
- .sessionTimeoutMs(zkClientConfig.getSessionTimeoutMillis())
- .connectionTimeoutMs(zkClientConfig.getConnectionTimeoutMillis())
- .retryPolicy(retryPolicy)
- .defaultData(new byte[0])
- .build();
+ .connectString(zkClientConfig.getConnectString())
+ .sessionTimeoutMs(zkClientConfig.getSessionTimeoutMillis())
+ .connectionTimeoutMs(zkClientConfig.getConnectionTimeoutMillis())
+ .retryPolicy(retryPolicy)
+ .defaultData(new byte[0])
+ .build();
curatorClient.start();
// We don't know what the heartbeats look like for the nodes, since we were just elected to monitoring
@@ -130,7 +129,7 @@ public void onStart() {
heartbeatMessages.clear();
for (final NodeIdentifier nodeId : clusterCoordinator.getNodeIdentifiers()) {
final NodeHeartbeat heartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(),
- clusterCoordinator.getConnectionStatus(nodeId), Collections.emptySet(), 0, 0L, 0, System.currentTimeMillis());
+ clusterCoordinator.getConnectionStatus(nodeId), Collections.emptySet(), 0, 0L, 0, System.currentTimeMillis());
heartbeatMessages.put(nodeId, heartbeat);
}
@@ -199,7 +198,6 @@ protected Set getClusterNodeIds() {
return new HashSet<>(clusterNodeIds.values());
}
-
@Override
public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException {
if (msg.getType() != MessageType.HEARTBEAT) {
@@ -220,7 +218,7 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio
final long systemStartTime = payload.getSystemStartTime();
final NodeHeartbeat nodeHeartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(),
- connectionStatus, roles, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime);
+ connectionStatus, roles, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime);
heartbeatMessages.put(heartbeat.getNodeIdentifier(), nodeHeartbeat);
logger.debug("Received new heartbeat from {}", nodeId);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
index 784eb2f7a8e1..7b1d8f696798 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.http;
import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger;
@@ -71,13 +70,25 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
+import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
public class StandardHttpResponseMerger implements HttpResponseMerger {
+
private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMerger.class);
- private static final List endpointMergers = new ArrayList<>();
- static {
+ private final List endpointMergers = new ArrayList<>();
+
+ public StandardHttpResponseMerger(final NiFiProperties nifiProperties) {
+ final String snapshotFrequency = nifiProperties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
+ long snapshotMillis;
+ try {
+ snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
+ } catch (final Exception e) {
+ snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
+ }
endpointMergers.add(new ControllerStatusEndpointMerger());
endpointMergers.add(new ControllerBulletinsEndpointMerger());
endpointMergers.add(new GroupStatusEndpointMerger());
@@ -108,7 +119,7 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
endpointMergers.add(new ListFlowFilesEndpointMerger());
endpointMergers.add(new ComponentStateEndpointMerger());
endpointMergers.add(new BulletinBoardEndpointMerger());
- endpointMergers.add(new StatusHistoryEndpointMerger());
+ endpointMergers.add(new StatusHistoryEndpointMerger(snapshotMillis));
endpointMergers.add(new SystemDiagnosticsEndpointMerger());
endpointMergers.add(new CountersEndpointMerger());
endpointMergers.add(new FlowMerger());
@@ -122,9 +133,6 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
endpointMergers.add(new FunnelsEndpointMerger());
}
- public StandardHttpResponseMerger() {
- }
-
@Override
public NodeResponse mergeResponses(final URI uri, final String httpMethod, final Set nodeResponses) {
if (nodeResponses.size() == 1) {
@@ -170,7 +178,6 @@ public NodeResponse mergeResponses(final URI uri, final String httpMethod, final
return response;
}
-
@Override
public Set getProblematicNodeResponses(final Set allResponses) {
// Check if there are any 2xx responses
@@ -190,7 +197,7 @@ public boolean isResponseInterpreted(final URI uri, final String httpMethod) {
return getEndpointResponseMerger(uri, httpMethod) != null;
}
- private static EndpointResponseMerger getEndpointResponseMerger(final URI uri, final String httpMethod) {
+ private EndpointResponseMerger getEndpointResponseMerger(final URI uri, final String httpMethod) {
return endpointMergers.stream().filter(p -> p.canHandle(uri, httpMethod)).findFirst().orElse(null);
}
@@ -198,13 +205,12 @@ private boolean hasSuccessfulResponse(final Set allResponses) {
return allResponses.stream().anyMatch(p -> p.is2xx());
}
-
private void drainResponses(final Set responses, final NodeResponse exclude) {
responses.stream()
- .parallel() // parallelize the draining of the responses, since we have multiple streams to consume
- .filter(response -> response != exclude) // don't include the explicitly excluded node
- .filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
- .forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
+ .parallel() // parallelize the draining of the responses, since we have multiple streams to consume
+ .filter(response -> response != exclude) // don't include the explicitly excluded node
+ .filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
+ .forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
}
private void drainResponse(final NodeResponse response) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMerger.java
index 507c1fb9ad56..ddd8759fa114 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMerger.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.http.endpoints;
import java.net.URI;
@@ -26,7 +25,6 @@
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
@@ -40,14 +38,13 @@
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
import org.apache.nifi.controller.status.history.StatusHistoryUtil;
import org.apache.nifi.controller.status.history.StatusSnapshot;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.status.NodeStatusSnapshotsDTO;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
public class StatusHistoryEndpointMerger implements EndpointResponseMerger {
+
public static final Pattern PROCESSOR_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/processors/[a-f0-9\\-]{36}/status/history");
public static final Pattern PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/status/history");
public static final Pattern REMOTE_PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/remote-process-groups/[a-f0-9\\-]{36}/status/history");
@@ -55,17 +52,8 @@ public class StatusHistoryEndpointMerger implements EndpointResponseMerger {
private final long componentStatusSnapshotMillis;
-
- public StatusHistoryEndpointMerger() {
- final NiFiProperties properties = NiFiProperties.getInstance();
- final String snapshotFrequency = properties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
- long snapshotMillis;
- try {
- snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
- } catch (final Exception e) {
- snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
- }
- componentStatusSnapshotMillis = snapshotMillis;
+ public StatusHistoryEndpointMerger(final long componentStatusSnapshotMillis) {
+ this.componentStatusSnapshotMillis = componentStatusSnapshotMillis;
}
private Map> getMetricDescriptors(final URI uri) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
index 07cefbbaf7c8..bd4e277804b2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
@@ -75,6 +75,7 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Function;
import java.util.stream.Collectors;
+import org.apache.nifi.util.NiFiProperties;
public class ThreadPoolRequestReplicator implements RequestReplicator {
@@ -107,10 +108,11 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
* @param clusterCoordinator the cluster coordinator to use for interacting with node statuses
* @param callback a callback that will be called whenever all of the responses have been gathered for a request. May be null.
* @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null.
+ * @param nifiProperties properties
*/
public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator,
- final RequestCompletionCallback callback, final EventReporter eventReporter) {
- this(numThreads, client, clusterCoordinator, "5 sec", "5 sec", callback, eventReporter);
+ final RequestCompletionCallback callback, final EventReporter eventReporter, final NiFiProperties nifiProperties) {
+ this(numThreads, client, clusterCoordinator, "5 sec", "5 sec", callback, eventReporter, nifiProperties);
}
/**
@@ -123,9 +125,11 @@ public ThreadPoolRequestReplicator(final int numThreads, final Client client, fi
* @param readTimeout the read timeout specified in milliseconds
* @param callback a callback that will be called whenever all of the responses have been gathered for a request. May be null.
* @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null.
+ * @param nifiProperties properties
*/
public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator,
- final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback, final EventReporter eventReporter) {
+ final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback,
+ final EventReporter eventReporter, final NiFiProperties nifiProperties) {
if (numThreads <= 0) {
throw new IllegalArgumentException("The number of threads must be greater than zero.");
} else if (client == null) {
@@ -136,7 +140,7 @@ public ThreadPoolRequestReplicator(final int numThreads, final Client client, fi
this.clusterCoordinator = clusterCoordinator;
this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
- this.responseMerger = new StandardHttpResponseMerger();
+ this.responseMerger = new StandardHttpResponseMerger(nifiProperties);
this.eventReporter = eventReporter;
this.callback = callback;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java
index daa3e5c8121b..18474615a556 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java
@@ -14,13 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.node;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.charset.StandardCharsets;
-import java.util.Properties;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.CuratorFramework;
@@ -33,6 +31,7 @@
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.controller.cluster.ZooKeeperClientConfig;
import org.apache.nifi.io.socket.SocketConfiguration;
+import org.apache.nifi.util.NiFiProperties;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
@@ -40,19 +39,20 @@
import org.slf4j.LoggerFactory;
/**
- * Uses Apache Curator to determine the address of the current cluster coordinator
+ * Uses Apache Curator to determine the address of the current cluster
+ * coordinator
*/
public class CuratorNodeProtocolSender extends AbstractNodeProtocolSender {
+
private static final Logger logger = LoggerFactory.getLogger(CuratorNodeProtocolSender.class);
private final String coordinatorPath;
private final ZooKeeperClientConfig zkConfig;
private InetSocketAddress coordinatorAddress;
-
- public CuratorNodeProtocolSender(final SocketConfiguration socketConfig, final ProtocolContext protocolContext, final Properties properties) {
+ public CuratorNodeProtocolSender(final SocketConfiguration socketConfig, final ProtocolContext protocolContext, final NiFiProperties nifiProperties) {
super(socketConfig, protocolContext);
- zkConfig = ZooKeeperClientConfig.createConfig(properties);
+ zkConfig = ZooKeeperClientConfig.createConfig(nifiProperties);
coordinatorPath = zkConfig.resolvePath("cluster/nodes/coordinator");
}
@@ -64,7 +64,7 @@ protected synchronized InetSocketAddress getServiceAddress() throws IOException
final RetryPolicy retryPolicy = new RetryNTimes(0, 0);
final CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(),
- zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy);
+ zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy);
curatorClient.start();
try {
@@ -85,7 +85,7 @@ public void process(final WatchedEvent event) {
final String[] splits = address.split(":");
if (splits.length != 2) {
final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates "
- + "that address is %s, but this is not in the expected format of :", address);
+ + "that address is %s, but this is not in the expected format of :", address);
logger.error(message);
throw new ProtocolException(message);
}
@@ -101,7 +101,7 @@ public void process(final WatchedEvent event) {
}
} catch (final NumberFormatException nfe) {
final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates "
- + "that address is %s, but the port is not a valid port number", address);
+ + "that address is %s, but the port is not a valid port number", address);
logger.error(message);
throw new ProtocolException(message);
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
index ac39dc58b07c..70338c1e37de 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.node;
import java.io.IOException;
@@ -25,7 +24,6 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Properties;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
@@ -72,6 +70,7 @@
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.services.FlowService;
+import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.revision.RevisionManager;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
@@ -80,6 +79,7 @@
import org.slf4j.LoggerFactory;
public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandler, RequestCompletionCallback {
+
private static final Logger logger = LoggerFactory.getLogger(NodeClusterCoordinator.class);
private static final String EVENT_CATEGORY = "Clustering";
@@ -97,6 +97,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
private final CuratorFramework curatorClient;
private final String nodesPathPrefix;
private final String coordinatorPath;
+ private final NiFiProperties nifiProperties;
private volatile FlowService flowService;
private volatile boolean connected;
@@ -107,18 +108,19 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
private final ConcurrentMap> nodeEvents = new ConcurrentHashMap<>();
public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter,
- final ClusterNodeFirewall firewall, final RevisionManager revisionManager, final Properties nifiProperties) {
+ final ClusterNodeFirewall firewall, final RevisionManager revisionManager, final NiFiProperties nifiProperties) {
this.senderListener = senderListener;
this.flowService = null;
this.eventReporter = eventReporter;
this.firewall = firewall;
this.revisionManager = revisionManager;
+ this.nifiProperties = nifiProperties;
final RetryPolicy retryPolicy = new RetryNTimes(10, 500);
final ZooKeeperClientConfig zkConfig = ZooKeeperClientConfig.createConfig(nifiProperties);
curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(),
- zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy);
+ zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy);
curatorClient.start();
nodesPathPrefix = zkConfig.resolvePath("cluster/nodes");
@@ -226,12 +228,15 @@ public void resetNodeStatuses(final Map st
}
/**
- * Attempts to update the nodeStatuses map by changing the value for the given node id from the current status to the new status, as in
- * ConcurrentMap.replace(nodeId, currentStatus, newStatus) but with the difference that this method can handle a null value
- * for currentStatus
+ * Attempts to update the nodeStatuses map by changing the value for the
+ * given node id from the current status to the new status, as in
+ * ConcurrentMap.replace(nodeId, currentStatus, newStatus) but with the
+ * difference that this method can handle a null value for
+ * currentStatus
*
* @param nodeId the node id
- * @param currentStatus the current status, or null if there is no value currently
+ * @param currentStatus the current status, or null if there is
+ * no value currently
* @param newStatus the new status to set
* @return true if the map was updated, false otherwise
*/
@@ -296,7 +301,6 @@ public void finishNodeConnection(final NodeIdentifier nodeId) {
updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED, getRoles(nodeId)));
}
-
@Override
public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
final Set connectedNodeIds = getNodeIdentifiers(NodeConnectionState.CONNECTED);
@@ -363,7 +367,6 @@ private NodeConnectionState getConnectionState(final NodeIdentifier nodeId) {
return status == null ? null : status.getState();
}
-
@Override
public Map> getConnectionStates() {
final Map> connectionStates = new HashMap<>();
@@ -521,18 +524,18 @@ public Set getNodeIdentifiers(final NodeConnectionState... state
}
return nodeStatuses.entrySet().stream()
- .filter(entry -> statesOfInterest.contains(entry.getValue().getState()))
- .map(entry -> entry.getKey())
- .collect(Collectors.toSet());
+ .filter(entry -> statesOfInterest.contains(entry.getValue().getState()))
+ .map(entry -> entry.getKey())
+ .collect(Collectors.toSet());
}
@Override
public NodeIdentifier getPrimaryNode() {
return nodeStatuses.values().stream()
- .filter(status -> status.getRoles().contains(ClusterRoles.PRIMARY_NODE))
- .findFirst()
- .map(status -> status.getNodeIdentifier())
- .orElse(null);
+ .filter(status -> status.getRoles().contains(ClusterRoles.PRIMARY_NODE))
+ .findFirst()
+ .map(status -> status.getNodeIdentifier())
+ .orElse(null);
}
@Override
@@ -582,13 +585,13 @@ private NodeIdentifier getElectedActiveCoordinatorNode(final boolean warnOnError
final Set connectedNodeIds = getNodeIdentifiers();
final NodeIdentifier electedNodeId = connectedNodeIds.stream()
- .filter(nodeId -> nodeId.getSocketAddress().equals(electedNodeHostname) && nodeId.getSocketPort() == electedNodePort)
- .findFirst()
- .orElse(null);
+ .filter(nodeId -> nodeId.getSocketAddress().equals(electedNodeHostname) && nodeId.getSocketPort() == electedNodePort)
+ .findFirst()
+ .orElse(null);
if (electedNodeId == null && warnOnError) {
logger.debug("Failed to determine which node is elected active Cluster Coordinator: ZooKeeper reports the address as {},"
- + "but there is no node with this address. Will attempt to communicate with node to determine its information", electedNodeAddress);
+ + "but there is no node with this address. Will attempt to communicate with node to determine its information", electedNodeAddress);
try {
final NodeConnectionStatus connectionStatus = senderListener.requestNodeConnectionStatus(electedNodeHostname, electedNodePort);
@@ -606,7 +609,7 @@ private NodeIdentifier getElectedActiveCoordinatorNode(final boolean warnOnError
}
} catch (final Exception e) {
logger.warn("Failed to determine which node is elected active Cluster Coordinator: ZooKeeper reports the address as {}, but there is no node with this address. "
- + "Attempted to determine the node's information but failed to retrieve its information due to {}", electedNodeAddress, e.toString());
+ + "Attempted to determine the node's information but failed to retrieve its information due to {}", electedNodeAddress, e.toString());
if (logger.isDebugEnabled()) {
logger.warn("", e);
@@ -656,8 +659,9 @@ private void addNodeEvent(final NodeIdentifier nodeId, final Severity severity,
}
/**
- * Updates the status of the node with the given ID to the given status and returns true
- * if successful, false if no node exists with the given ID
+ * Updates the status of the node with the given ID to the given status and
+ * returns true if successful, false if no node
+ * exists with the given ID
*
* @param status the new status of the node
*/
@@ -705,7 +709,8 @@ void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus) {
* Notifies other nodes that the status of a node changed
*
* @param updatedStatus the updated status for a node in the cluster
- * @param notifyAllNodes if true will notify all nodes. If false, will notify only the cluster coordinator
+ * @param notifyAllNodes if true will notify all nodes. If
+ * false, will notify only the cluster coordinator
*/
void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus, final boolean notifyAllNodes, final boolean waitForCoordinator) {
// If this node is the active cluster coordinator, then we are going to replicate to all nodes.
@@ -770,7 +775,7 @@ public void run() {
Thread.sleep(100L);
} catch (final InterruptedException ie) {
logger.info("Could not send Reconnection request to {} because thread was "
- + "interrupted before FlowService was made available", request.getNodeId());
+ + "interrupted before FlowService was made available", request.getNodeId());
Thread.currentThread().interrupt();
return;
}
@@ -797,7 +802,7 @@ public void run() {
} catch (final Exception e) {
logger.warn("Problem encountered issuing reconnection request to node " + request.getNodeId(), e);
eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Problem encountered issuing reconnection request to node "
- + request.getNodeId() + " due to: " + e);
+ + request.getNodeId() + " due to: " + e);
}
try {
@@ -810,7 +815,7 @@ public void run() {
// We failed to reconnect too many times. We must now mark node as disconnected.
if (NodeConnectionState.CONNECTING == getConnectionState(request.getNodeId())) {
requestNodeDisconnect(request.getNodeId(), DisconnectionCode.UNABLE_TO_COMMUNICATE,
- "Attempted to request that node reconnect to cluster but could not communicate with node");
+ "Attempted to request that node reconnect to cluster but could not communicate with node");
}
}
}, "Reconnect " + request.getNodeId());
@@ -944,10 +949,10 @@ private NodeIdentifier resolveNodeId(final NodeIdentifier proposedIdentifier) {
} else {
// there is a node with that ID and it's a different node
resolvedNodeId = new NodeIdentifier(UUID.randomUUID().toString(), proposedIdentifier.getApiAddress(), proposedIdentifier.getApiPort(),
- proposedIdentifier.getSocketAddress(), proposedIdentifier.getSocketPort(), proposedIdentifier.getSiteToSiteAddress(),
- proposedIdentifier.getSiteToSitePort(), proposedIdentifier.getSiteToSiteHttpApiPort(), proposedIdentifier.isSiteToSiteSecure());
+ proposedIdentifier.getSocketAddress(), proposedIdentifier.getSocketPort(), proposedIdentifier.getSiteToSiteAddress(),
+ proposedIdentifier.getSiteToSitePort(), proposedIdentifier.getSiteToSiteHttpApiPort(), proposedIdentifier.isSiteToSiteSecure());
logger.debug("A node already exists with ID {}. Proposed Node Identifier was {}; existing Node Identifier is {}; Resolved Node Identifier is {}",
- proposedIdentifier.getId(), proposedIdentifier, getNodeIdentifier(proposedIdentifier.getId()), resolvedNodeId);
+ proposedIdentifier.getId(), proposedIdentifier, getNodeIdentifier(proposedIdentifier.getId()), resolvedNodeId);
}
return resolvedNodeId;
@@ -989,7 +994,7 @@ private ConnectionResponse createConnectionResponse(final ConnectionRequest requ
dataFlow = flowService.createDataFlow();
} catch (final IOException ioe) {
logger.error("Unable to obtain current dataflow from FlowService in order to provide the flow to "
- + resolvedNodeIdentifier + ". Will tell node to try again later", ioe);
+ + resolvedNodeIdentifier + ". Will tell node to try again later", ioe);
}
}
@@ -998,37 +1003,37 @@ private ConnectionResponse createConnectionResponse(final ConnectionRequest requ
// the flow management service a chance to retrieve a current flow
final int tryAgainSeconds = 5;
addNodeEvent(resolvedNodeIdentifier, Severity.WARNING, "Connection requested from node, but manager was unable to obtain current flow. "
- + "Instructing node to try again in " + tryAgainSeconds + " seconds.");
+ + "Instructing node to try again in " + tryAgainSeconds + " seconds.");
// return try later response
return new ConnectionResponse(tryAgainSeconds);
}
return new ConnectionResponse(resolvedNodeIdentifier, dataFlow, instanceId, new ArrayList<>(nodeStatuses.values()),
- revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList()));
+ revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList()));
}
private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) {
return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort(),
- nodeId.getSocketAddress(), nodeId.getSocketPort(),
- nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(),
- nodeId.getSiteToSiteHttpApiPort(), nodeId.isSiteToSiteSecure(), dn);
+ nodeId.getSocketAddress(), nodeId.getSocketPort(),
+ nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(),
+ nodeId.getSiteToSiteHttpApiPort(), nodeId.isSiteToSiteSecure(), dn);
}
@Override
public boolean canHandle(final ProtocolMessage msg) {
return MessageType.CONNECTION_REQUEST == msg.getType() || MessageType.NODE_STATUS_CHANGE == msg.getType()
- || MessageType.NODE_CONNECTION_STATUS_REQUEST == msg.getType();
+ || MessageType.NODE_CONNECTION_STATUS_REQUEST == msg.getType();
}
private boolean isMutableRequest(final String method) {
return "DELETE".equalsIgnoreCase(method) || "POST".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method);
}
-
/**
- * Callback that is called after an HTTP Request has been replicated to nodes in the cluster.
- * This allows us to disconnect nodes that did not complete the request, if applicable.
+ * Callback that is called after an HTTP Request has been replicated to
+ * nodes in the cluster. This allows us to disconnect nodes that did not
+ * complete the request, if applicable.
*/
@Override
public void afterRequest(final String uriPath, final String method, final Set nodeResponses) {
@@ -1047,7 +1052,7 @@ public void afterRequest(final String uriPath, final String method, final Set problematicNodeResponses = responseMerger.getProblematicNodeResponses(nodeResponses);
// all nodes failed
@@ -1055,7 +1060,7 @@ public void afterRequest(final String uriPath, final String method, final Set problematicNodeResponses, final String uriPath) {
if (COUNTER_URI_PATTERN.matcher(uriPath).matches()) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.java
index fc0eaf2658c7..31c3b1d43309 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.java
@@ -33,24 +33,24 @@
public class ThreadPoolRequestReplicatorFactoryBean implements FactoryBean, ApplicationContextAware {
private ApplicationContext applicationContext;
- private NiFiProperties properties;
+ private NiFiProperties nifiProperties;
private ThreadPoolRequestReplicator replicator = null;
@Override
public ThreadPoolRequestReplicator getObject() throws Exception {
- if (replicator == null && properties.isNode()) {
+ if (replicator == null && nifiProperties.isNode()) {
final EventReporter eventReporter = applicationContext.getBean("eventReporter", EventReporter.class);
final ClusterCoordinator clusterCoordinator = applicationContext.getBean("clusterCoordinator", ClusterCoordinator.class);
final RequestCompletionCallback requestCompletionCallback = applicationContext.getBean("clusterCoordinator", RequestCompletionCallback.class);
- final int numThreads = properties.getClusterNodeProtocolThreads();
- final Client jerseyClient = WebUtils.createClient(new DefaultClientConfig(), SslContextFactory.createSslContext(properties));
- final String connectionTimeout = properties.getClusterNodeConnectionTimeout();
- final String readTimeout = properties.getClusterNodeReadTimeout();
+ final int numThreads = nifiProperties.getClusterNodeProtocolThreads();
+ final Client jerseyClient = WebUtils.createClient(new DefaultClientConfig(), SslContextFactory.createSslContext(nifiProperties));
+ final String connectionTimeout = nifiProperties.getClusterNodeConnectionTimeout();
+ final String readTimeout = nifiProperties.getClusterNodeReadTimeout();
replicator = new ThreadPoolRequestReplicator(numThreads, jerseyClient, clusterCoordinator,
- connectionTimeout, readTimeout, requestCompletionCallback, eventReporter);
+ connectionTimeout, readTimeout, requestCompletionCallback, eventReporter, nifiProperties);
}
return replicator;
@@ -71,8 +71,8 @@ public void setApplicationContext(ApplicationContext applicationContext) throws
this.applicationContext = applicationContext;
}
- public void setProperties(NiFiProperties properties) {
- this.properties = properties;
+ public void setProperties(final NiFiProperties properties) {
+ this.nifiProperties = properties;
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
index 5086dc004dfc..2307538e4395 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
@@ -27,7 +27,6 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Properties;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
@@ -52,6 +51,7 @@ public class TestAbstractHeartbeatMonitor {
@Before
public void setup() throws Exception {
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties");
nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", null, null, false);
}
@@ -179,10 +179,10 @@ private TestFriendlyHeartbeatMonitor createMonitor(final ClusterCoordinator coor
return monitor;
}
- private Properties createProperties() {
- final Properties properties = new Properties();
- properties.setProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, "10 ms");
- return properties;
+ private NiFiProperties createProperties() {
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, "10 ms");
+ return NiFiProperties.createBasicNiFiProperties(null, addProps);
}
private static class ClusterCoordinatorAdapter implements ClusterCoordinator {
@@ -328,8 +328,8 @@ private static class TestFriendlyHeartbeatMonitor extends AbstractHeartbeatMonit
private Map heartbeats = new HashMap<>();
private final Object mutex = new Object();
- public TestFriendlyHeartbeatMonitor(ClusterCoordinator clusterCoordinator, Properties properties) {
- super(clusterCoordinator, properties);
+ public TestFriendlyHeartbeatMonitor(ClusterCoordinator clusterCoordinator, NiFiProperties nifiProperties) {
+ super(clusterCoordinator, nifiProperties);
}
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy
index bd9b265527d3..03aa08ad0d60 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy
@@ -46,7 +46,8 @@ import spock.lang.Unroll
class StandardHttpResponseMergerSpec extends Specification {
def setup() {
- System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties"
+ def propFile = StandardHttpResponseMergerSpec.class.getResource("/conf/nifi.properties").getFile()
+ System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile
}
def cleanup() {
@@ -55,7 +56,7 @@ class StandardHttpResponseMergerSpec extends Specification {
def "MergeResponses: mixed HTTP GET response statuses, expecting #expectedStatus"() {
given:
- def responseMerger = new StandardHttpResponseMerger()
+ def responseMerger = new StandardHttpResponseMerger(NiFiProperties.createBasicNiFiProperties(null,null))
def requestUri = new URI('http://server/resource')
def requestId = UUID.randomUUID().toString()
def Map> mockToRequestEntity = [:]
@@ -93,7 +94,7 @@ class StandardHttpResponseMergerSpec extends Specification {
mapper.setSerializationConfig(serializationConfig.withSerializationInclusion(JsonSerialize.Inclusion.NON_NULL).withAnnotationIntrospector(jaxbIntrospector));
and: "setup of the data to be used in the test"
- def responseMerger = new StandardHttpResponseMerger()
+ def responseMerger = new StandardHttpResponseMerger(NiFiProperties.createBasicNiFiProperties(null,null))
def requestUri = new URI("http://server/$requestUriPart")
def requestId = UUID.randomUUID().toString()
def Map mockToRequestEntity = [:]
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMergerSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMergerSpec.groovy
index 69dd82a9a501..350269d7f614 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMergerSpec.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/StatusHistoryEndpointMergerSpec.groovy
@@ -32,7 +32,8 @@ import spock.lang.Unroll
class StatusHistoryEndpointMergerSpec extends Specification {
def setup() {
- System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties"
+ def propFile = StatusHistoryEndpointMergerSpec.class.getResource("/conf/nifi.properties").getFile()
+ System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile
}
def cleanup() {
@@ -48,7 +49,7 @@ class StatusHistoryEndpointMergerSpec extends Specification {
mapper.setSerializationConfig(serializationConfig.withSerializationInclusion(JsonSerialize.Inclusion.NON_NULL).withAnnotationIntrospector(jaxbIntrospector));
and: "setup of the data to be used in the test"
- def merger = new StatusHistoryEndpointMerger()
+ def merger = new StatusHistoryEndpointMerger(2)
def requestUri = new URI("http://server/$requestUriPart")
def requestId = UUID.randomUUID().toString()
def Map mockToRequestEntity = [:]
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
index 5eac84616caa..ebefce2eb7c7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.http.replication;
import static org.junit.Assert.assertEquals;
@@ -75,8 +74,9 @@ public static void setupClass() {
}
/**
- * If we replicate a request, whenever we obtain the merged response from the AsyncClusterResponse object,
- * the response should no longer be available and should be cleared from internal state. This test is to
+ * If we replicate a request, whenever we obtain the merged response from
+ * the AsyncClusterResponse object, the response should no longer be
+ * available and should be cleared from internal state. This test is to
* verify that this behavior occurs.
*/
@Test
@@ -105,7 +105,6 @@ public void testResponseRemovedWhenCompletedAndFetched() {
});
}
-
@Test(timeout = 15000)
public void testLongWaitForResponse() {
withReplicator(replicator -> {
@@ -132,7 +131,7 @@ public void testLongWaitForResponse() {
assertTrue(response.isComplete());
assertNotNull(response.getMergedResponse());
assertNull(replicator.getClusterResponse(response.getRequestIdentifier()));
- } , Status.OK, 1000, new ClientHandlerException(new SocketTimeoutException()));
+ }, Status.OK, 1000, new ClientHandlerException(new SocketTimeoutException()));
}
@Test(timeout = 15000)
@@ -153,10 +152,9 @@ public void testCompleteOnError() {
final AsyncClusterResponse response = replicator.replicate(nodeIds, HttpMethod.GET, uri, entity, new HashMap<>(), true, true);
assertNotNull(response.awaitMergedResponse(1, TimeUnit.SECONDS));
- } , null, 0L, new IllegalArgumentException("Exception created for unit test"));
+ }, null, 0L, new IllegalArgumentException("Exception created for unit test"));
}
-
@Test(timeout = 15000)
public void testMultipleRequestWithTwoPhaseCommit() {
final Set nodeIds = new HashSet<>();
@@ -167,7 +165,8 @@ public void testMultipleRequestWithTwoPhaseCommit() {
Mockito.when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenReturn(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED, Collections.emptySet()));
final AtomicInteger requestCount = new AtomicInteger(0);
- final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) {
+ final ThreadPoolRequestReplicator replicator
+ = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null, NiFiProperties.createBasicNiFiProperties(null, null)) {
@Override
protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) {
// the resource builder will not expose its headers to us, so we are using Mockito's Whitebox class to extract them.
@@ -191,7 +190,7 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde
try {
final AsyncClusterResponse clusterResponse = replicator.replicate(nodeIds, HttpMethod.POST,
- new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>(), true, true);
+ new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>(), true, true);
clusterResponse.awaitMergedResponse();
// Ensure that we received two requests - the first should contain the X-NcmExpects header; the second should not.
@@ -233,7 +232,8 @@ public void testMutableRequestRequiresAllNodesConnected() throws URISyntaxExcept
nodeMap.put(NodeConnectionState.CONNECTING, otherState);
Mockito.when(coordinator.getConnectionStates()).thenReturn(nodeMap);
- final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) {
+ final ThreadPoolRequestReplicator replicator
+ = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null, NiFiProperties.createBasicNiFiProperties(null, null)) {
@Override
public AsyncClusterResponse replicate(Set nodeIds, String method, URI uri, Object entity, Map headers,
boolean indicateReplicated, boolean verify) {
@@ -278,7 +278,6 @@ public AsyncClusterResponse replicate(Set nodeIds, String method
}
}
-
@Test(timeout = 15000)
public void testOneNodeRejectsTwoPhaseCommit() {
final Set nodeIds = new HashSet<>();
@@ -287,7 +286,8 @@ public void testOneNodeRejectsTwoPhaseCommit() {
final ClusterCoordinator coordinator = createClusterCoordinator();
final AtomicInteger requestCount = new AtomicInteger(0);
- final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) {
+ final ThreadPoolRequestReplicator replicator
+ = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null, NiFiProperties.createBasicNiFiProperties(null, null)) {
@Override
protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) {
// the resource builder will not expose its headers to us, so we are using Mockito's Whitebox class to extract them.
@@ -309,7 +309,7 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde
try {
final AsyncClusterResponse clusterResponse = replicator.replicate(nodeIds, HttpMethod.POST,
- new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>(), true, true);
+ new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>(), true, true);
clusterResponse.awaitMergedResponse();
Assert.fail("Expected to get an IllegalClusterStateException but did not");
@@ -322,15 +322,14 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde
}
}
-
-
private void withReplicator(final WithReplicator function) {
withReplicator(function, ClientResponse.Status.OK, 0L, null);
}
private void withReplicator(final WithReplicator function, final Status status, final long delayMillis, final RuntimeException failure) {
final ClusterCoordinator coordinator = createClusterCoordinator();
- final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) {
+ final ThreadPoolRequestReplicator replicator
+ = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null, NiFiProperties.createBasicNiFiProperties(null, null)) {
@Override
protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) {
if (delayMillis > 0L) {
@@ -362,6 +361,7 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde
}
private interface WithReplicator {
+
void withReplicator(ThreadPoolRequestReplicator replicator) throws Exception;
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java
index 2f034b34b81b..aaa9dca3290b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.coordination.node;
import static org.junit.Assert.assertEquals;
@@ -27,9 +26,9 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Properties;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
@@ -47,6 +46,7 @@
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.services.FlowService;
+import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.revision.RevisionManager;
import org.junit.Assert;
import org.junit.Before;
@@ -56,18 +56,21 @@
import org.mockito.stubbing.Answer;
public class TestNodeClusterCoordinator {
+
private NodeClusterCoordinator coordinator;
private ClusterCoordinationProtocolSenderListener senderListener;
private List nodeStatuses;
- private Properties createProperties() {
- final Properties props = new Properties();
- props.put("nifi.zookeeper.connect.string", "localhost:2181");
- return props;
+ private NiFiProperties createProperties() {
+ final Map addProps = new HashMap<>();
+ addProps.put("nifi.zookeeper.connect.string", "localhost:2181");
+ return NiFiProperties.createBasicNiFiProperties(null, addProps);
}
@Before
public void setup() throws IOException {
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties");
+
senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class);
nodeStatuses = Collections.synchronizedList(new ArrayList<>());
@@ -113,7 +116,7 @@ public void testConnectionResponseIndicatesAllNodes() throws IOException {
assertNotNull(statuses);
assertEquals(6, statuses.size());
final Map statusMap = statuses.stream().collect(
- Collectors.toMap(status -> status.getNodeIdentifier(), status -> status));
+ Collectors.toMap(status -> status.getNodeIdentifier(), status -> status));
assertEquals(DisconnectionCode.LACK_OF_HEARTBEAT, statusMap.get(createNodeId(1)).getDisconnectCode());
assertEquals(NodeConnectionState.DISCONNECTING, statusMap.get(createNodeId(2)).getState());
@@ -258,7 +261,6 @@ public void testStatusChangesReplicated() throws InterruptedException, IOExcepti
assertEquals("Unit Test", statusChange.getDisconnectReason());
}
-
@Test
public void testGetConnectionStates() throws IOException {
// Add a disconnected node
@@ -316,7 +318,6 @@ public void testGetNodeIdentifiers() throws IOException {
assertTrue(disconnectedIds.contains(createNodeId(1)));
}
-
@Test(timeout = 5000)
public void testRequestNodeDisconnect() throws InterruptedException {
// Add a connected node
@@ -341,7 +342,6 @@ public void testRequestNodeDisconnect() throws InterruptedException {
assertEquals(NodeConnectionState.DISCONNECTED, status.getState());
}
-
@Test(timeout = 5000)
public void testCannotDisconnectLastNode() throws InterruptedException {
// Add a connected node
@@ -369,7 +369,6 @@ public void testCannotDisconnectLastNode() throws InterruptedException {
coordinator.requestNodeDisconnect(nodeId2, DisconnectionCode.USER_DISCONNECTED, "Unit Test");
}
-
@Test(timeout = 5000)
public void testUpdateNodeStatusOutOfOrder() throws InterruptedException {
// Add a connected node
@@ -386,7 +385,7 @@ public void testUpdateNodeStatusOutOfOrder() throws InterruptedException {
nodeStatuses.clear();
final NodeConnectionStatus oldStatus = new NodeConnectionStatus(-1L, nodeId1, NodeConnectionState.DISCONNECTED,
- DisconnectionCode.BLOCKED_BY_FIREWALL, null, 0L, null);
+ DisconnectionCode.BLOCKED_BY_FIREWALL, null, 0L, null);
final NodeStatusChangeMessage msg = new NodeStatusChangeMessage();
msg.setNodeId(nodeId1);
msg.setNodeConnectionStatus(oldStatus);
@@ -452,7 +451,6 @@ public void testUpdateNodeRoles() throws InterruptedException {
assertEquals(Collections.singleton(ClusterRoles.PRIMARY_NODE), id2Msg.getRoles());
}
-
@Test
public void testProposedIdentifierResolvedIfConflict() {
final NodeIdentifier id1 = new NodeIdentifier("1234", "localhost", 8000, "localhost", 9000, "localhost", 10000, 11000, false);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
index dbd8c004113b..58096250e533 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
@@ -18,7 +18,9 @@
package org.apache.nifi.cluster.integration;
import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@@ -91,11 +93,11 @@ public Set getNodes() {
public Node createNode() {
- NiFiProperties.getInstance().setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, getZooKeeperConnectString());
- NiFiProperties.getInstance().setProperty(NiFiProperties.CLUSTER_IS_NODE, "true");
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.ZOOKEEPER_CONNECT_STRING, getZooKeeperConnectString());
+ addProps.put(NiFiProperties.CLUSTER_IS_NODE, "true");
- final NiFiProperties properties = NiFiProperties.getInstance().copy();
- final Node node = new Node(properties);
+ final Node node = new Node(NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", addProps));
node.start();
nodes.add(node);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java
index 5bfe83c6383b..899f3122a69b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java
@@ -21,6 +21,7 @@
import java.net.ServerSocket;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.UUID;
@@ -90,10 +91,26 @@ public Node(final NiFiProperties properties) {
public Node(final NodeIdentifier nodeId, final NiFiProperties properties) {
this.nodeId = nodeId;
- this.nodeProperties = properties;
+ this.nodeProperties = new NiFiProperties() {
+ @Override
+ public String getProperty(String key) {
+ if(key.equals(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT)){
+ return String.valueOf(nodeId.getSocketPort());
+ }else if(key.equals(NiFiProperties.WEB_HTTP_PORT)){
+ return String.valueOf(nodeId.getApiPort());
+ }else {
+ return properties.getProperty(key);
+ }
+ }
- nodeProperties.setProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT, String.valueOf(nodeId.getSocketPort()));
- nodeProperties.setProperty(NiFiProperties.WEB_HTTP_PORT, String.valueOf(nodeId.getApiPort()));
+ @Override
+ public Set getPropertyKeys() {
+ final Set keys = new HashSet<>(properties.getPropertyKeys());
+ keys.add(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT);
+ keys.add(NiFiProperties.WEB_HTTP_PORT);
+ return keys;
+ }
+ };
revisionManager = Mockito.mock(RevisionManager.class);
Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections. emptyList());
@@ -110,7 +127,7 @@ public synchronized void start() {
final HeartbeatMonitor heartbeatMonitor = createHeartbeatMonitor();
flowController = FlowController.createClusteredInstance(Mockito.mock(FlowFileEventRepository.class), nodeProperties,
- null, null, StringEncryptor.createEncryptor(), protocolSender, Mockito.mock(BulletinRepository.class), clusterCoordinator, heartbeatMonitor, VariableRegistry.EMPTY_REGISTRY);
+ null, null, StringEncryptor.createEncryptor(nodeProperties), protocolSender, Mockito.mock(BulletinRepository.class), clusterCoordinator, heartbeatMonitor, VariableRegistry.EMPTY_REGISTRY);
try {
flowController.initializeFlow();
@@ -123,7 +140,7 @@ public synchronized void start() {
flowController.getStateManagerProvider().getStateManager("Cluster Node Configuration").setState(Collections.singletonMap("Node UUID", nodeId.getId()), Scope.LOCAL);
flowService = StandardFlowService.createClusteredInstance(flowController, nodeProperties, senderListener, clusterCoordinator,
- StringEncryptor.createEncryptor(), revisionManager, Mockito.mock(Authorizer.class));
+ StringEncryptor.createEncryptor(nodeProperties), revisionManager, Mockito.mock(Authorizer.class));
flowService.start();
flowService.load(null);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
index 4f860016c5dd..43660688c722 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
@@ -134,10 +134,20 @@ public interface ServerProtocol extends VersionedRemoteResource {
*
* @param peer peer
* @param clusterNodeInfo the cluster information
+ * @param remoteInputHost the remote input host
+ * @param remoteInputPort the remote input port
+ * @param remoteInputHttpPort the remote input http port
+ * @param isSiteToSiteSecure whether site to site is secure
*
* @throws java.io.IOException ioe
*/
- void sendPeerList(Peer peer, Optional clusterNodeInfo) throws IOException;
+ void sendPeerList(
+ Peer peer,
+ Optional clusterNodeInfo,
+ String remoteInputHost,
+ int remoteInputPort,
+ int remoteInputHttpPort,
+ boolean isSiteToSiteSecure) throws IOException;
void shutdown(Peer peer);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java
index 5ad9a3c482c4..2a0f0dec102e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/connectable/StandardConnection.java
@@ -40,7 +40,6 @@
import org.apache.nifi.processor.FlowFileFilter;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.ProvenanceEventRepository;
-import org.apache.nifi.util.NiFiProperties;
import java.util.ArrayList;
import java.util.Collection;
@@ -55,7 +54,9 @@
import java.util.stream.Collectors;
/**
- * Models a connection between connectable components. A connection may contain one or more relationships that map the source component to the destination component.
+ * Models a connection between connectable components. A connection may contain
+ * one or more relationships that map the source component to the destination
+ * component.
*/
public final class StandardConnection implements Connection {
@@ -82,7 +83,7 @@ private StandardConnection(final Builder builder) {
relationships = new AtomicReference<>(Collections.unmodifiableCollection(builder.relationships));
scheduler = builder.scheduler;
flowFileQueue = new StandardFlowFileQueue(id, this, builder.flowFileRepository, builder.provenanceRepository, builder.resourceClaimManager,
- scheduler, builder.swapManager, builder.eventReporter, NiFiProperties.getInstance().getQueueSwapThreshold());
+ scheduler, builder.swapManager, builder.eventReporter, builder.queueSwapThreshold);
hashCode = new HashCodeBuilder(7, 67).append(id).toHashCode();
}
@@ -307,8 +308,10 @@ public String toString() {
}
/**
- * Gives this Connection ownership of the given FlowFile and allows the Connection to hold on to the FlowFile but NOT provide the FlowFile to consumers. This allows us to ensure that the
- * Connection is not deleted during the middle of a Session commit.
+ * Gives this Connection ownership of the given FlowFile and allows the
+ * Connection to hold on to the FlowFile but NOT provide the FlowFile to
+ * consumers. This allows us to ensure that the Connection is not deleted
+ * during the middle of a Session commit.
*
* @param flowFile to add
*/
@@ -338,6 +341,7 @@ public static class Builder {
private FlowFileRepository flowFileRepository;
private ProvenanceEventRepository provenanceRepository;
private ResourceClaimManager resourceClaimManager;
+ private int queueSwapThreshold;
public Builder(final ProcessScheduler scheduler) {
this.scheduler = scheduler;
@@ -409,6 +413,11 @@ public Builder resourceClaimManager(final ResourceClaimManager resourceClaimMana
return this;
}
+ public Builder queueSwapThreshold(final int queueSwapThreshold) {
+ this.queueSwapThreshold = queueSwapThreshold;
+ return this;
+ }
+
public StandardConnection build() {
if (source == null) {
throw new IllegalStateException("Cannot build a Connection without a Source");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
index a4c267c1db34..a61d7fe96704 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
@@ -63,7 +63,8 @@
/**
*
- * An implementation of the {@link FlowFileSwapManager} that swaps FlowFiles to/from local disk
+ * An implementation of the {@link FlowFileSwapManager} that swaps FlowFiles
+ * to/from local disk
*
*/
public class FileSystemSwapManager implements FlowFileSwapManager {
@@ -83,9 +84,15 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
private EventReporter eventReporter;
private ResourceClaimManager claimManager;
+ /**
+ * Default no args constructor for service loading only.
+ */
public FileSystemSwapManager() {
- final NiFiProperties properties = NiFiProperties.getInstance();
- final Path flowFileRepoPath = properties.getFlowFileRepositoryPath();
+ storageDirectory = null;
+ }
+
+ public FileSystemSwapManager(final NiFiProperties nifiProperties) {
+ final Path flowFileRepoPath = nifiProperties.getFlowFileRepositoryPath();
this.storageDirectory = flowFileRepoPath.resolve("swap").toFile();
if (!storageDirectory.exists() && !storageDirectory.mkdirs()) {
@@ -93,7 +100,6 @@ public FileSystemSwapManager() {
}
}
-
@Override
public synchronized void initialize(final SwapManagerInitializationContext initializationContext) {
this.claimManager = initializationContext.getResourceClaimManager();
@@ -129,7 +135,6 @@ public String swapOut(final List toSwap, final FlowFileQueue flo
return swapLocation;
}
-
@Override
public SwapContents swapIn(final String swapLocation, final FlowFileQueue flowFileQueue) throws IOException {
final File swapFile = new File(swapLocation);
@@ -152,15 +157,14 @@ public SwapContents peek(final String swapLocation, final FlowFileQueue flowFile
final SwapContents swapContents;
try (final InputStream fis = new FileInputStream(swapFile);
- final InputStream bis = new BufferedInputStream(fis);
- final DataInputStream in = new DataInputStream(bis)) {
+ final InputStream bis = new BufferedInputStream(fis);
+ final DataInputStream in = new DataInputStream(bis)) {
swapContents = deserializeFlowFiles(in, swapLocation, flowFileQueue, claimManager);
}
return swapContents;
}
-
@Override
public void purge() {
final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() {
@@ -177,7 +181,6 @@ public boolean accept(final File dir, final String name) {
}
}
-
@Override
public List recoverSwapLocations(final FlowFileQueue flowFileQueue) throws IOException {
final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() {
@@ -217,13 +220,13 @@ public boolean accept(final File dir, final String name) {
// Read the queue identifier from the swap file to check if the swap file is for this queue
try (final InputStream fis = new FileInputStream(swapFile);
- final InputStream bufferedIn = new BufferedInputStream(fis);
- final DataInputStream in = new DataInputStream(bufferedIn)) {
+ final InputStream bufferedIn = new BufferedInputStream(fis);
+ final DataInputStream in = new DataInputStream(bufferedIn)) {
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
final String errMsg = "Cannot swap FlowFiles in from " + swapFile + " because the encoding version is "
- + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
+ + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, errMsg);
throw new IOException(errMsg);
@@ -246,13 +249,13 @@ public SwapSummary getSwapSummary(final String swapLocation) throws IOException
// read record from disk via the swap file
try (final InputStream fis = new FileInputStream(swapFile);
- final InputStream bufferedIn = new BufferedInputStream(fis);
- final DataInputStream in = new DataInputStream(bufferedIn)) {
+ final InputStream bufferedIn = new BufferedInputStream(fis);
+ final DataInputStream in = new DataInputStream(bufferedIn)) {
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
final String errMsg = "Cannot swap FlowFiles in from " + swapFile + " because the encoding version is "
- + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
+ + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)";
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, errMsg);
throw new IOException(errMsg);
@@ -348,7 +351,7 @@ public static int serializeFlowFiles(final List toSwap, final Fl
out.flush();
}
- logger.info("Successfully swapped out {} FlowFiles from {} to Swap File {}", new Object[] {toSwap.size(), queue, swapLocation});
+ logger.info("Successfully swapped out {} FlowFiles from {} to Swap File {}", new Object[]{toSwap.size(), queue, swapLocation});
return toSwap.size();
}
@@ -376,13 +379,13 @@ static SwapContents deserializeFlowFiles(final DataInputStream in, final String
final int swapEncodingVersion = in.readInt();
if (swapEncodingVersion > SWAP_ENCODING_VERSION) {
throw new IOException("Cannot swap FlowFiles in from SwapFile because the encoding version is "
- + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)");
+ + swapEncodingVersion + ", which is too new (expecting " + SWAP_ENCODING_VERSION + " or less)");
}
final String connectionId = in.readUTF(); // Connection ID
if (!connectionId.equals(queue.getIdentifier())) {
- throw new IllegalArgumentException("Cannot deserialize FlowFiles from Swap File at location " + swapLocation +
- " because those FlowFiles belong to Connection with ID " + connectionId + " and an attempt was made to swap them into a Connection with ID " + queue.getIdentifier());
+ throw new IllegalArgumentException("Cannot deserialize FlowFiles from Swap File at location " + swapLocation
+ + " because those FlowFiles belong to Connection with ID " + connectionId + " and an attempt was made to swap them into a Connection with ID " + queue.getIdentifier());
}
int numRecords = 0;
@@ -396,8 +399,8 @@ static SwapContents deserializeFlowFiles(final DataInputStream in, final String
}
} catch (final EOFException eof) {
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
- final SwapSummary summary = new StandardSwapSummary(queueSize, maxRecordId, Collections. emptyList());
- final SwapContents partialContents = new StandardSwapContents(summary, Collections. emptyList());
+ final SwapSummary summary = new StandardSwapSummary(queueSize, maxRecordId, Collections.emptyList());
+ final SwapContents partialContents = new StandardSwapContents(summary, Collections.emptyList());
throw new IncompleteSwapFileException(swapLocation, partialContents);
}
@@ -406,7 +409,7 @@ static SwapContents deserializeFlowFiles(final DataInputStream in, final String
}
private static SwapContents deserializeFlowFiles(final DataInputStream in, final QueueSize queueSize, final Long maxRecordId,
- final int serializationVersion, final boolean incrementContentClaims, final ResourceClaimManager claimManager, final String location) throws IOException {
+ final int serializationVersion, final boolean incrementContentClaims, final ResourceClaimManager claimManager, final String location) throws IOException {
final List flowFiles = new ArrayList<>(queueSize.getObjectCount());
final List resourceClaims = new ArrayList<>(queueSize.getObjectCount());
Long maxId = maxRecordId;
@@ -432,7 +435,7 @@ private static SwapContents deserializeFlowFiles(final DataInputStream in, final
if (serializationVersion > 1) {
// Lineage information was added in version 2
- if(serializationVersion < 10){
+ if (serializationVersion < 10) {
final int numLineageIdentifiers = in.readInt();
for (int lineageIdIdx = 0; lineageIdIdx < numLineageIdentifiers; lineageIdIdx++) {
in.readUTF(); //skip each identifier
@@ -590,7 +593,6 @@ private static void fillBuffer(final InputStream in, final byte[] buffer, final
}
}
-
private void error(final String error) {
logger.error(error);
if (eventReporter != null) {
@@ -605,9 +607,8 @@ private void warn(final String warning) {
}
}
-
-
private static class SwapFileComparator implements Comparator {
+
@Override
public int compare(final String o1, final String o2) {
if (o1 == o2) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 77c3dd7b5f01..abb4b78bc2b4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -275,7 +275,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private final SnippetManager snippetManager;
private final long gracefulShutdownSeconds;
private final ExtensionManager extensionManager;
- private final NiFiProperties properties;
+ private final NiFiProperties nifiProperties;
private final SSLContext sslContext;
private final Set externalSiteListeners = new HashSet<>();
private final AtomicReference counterRepositoryRef;
@@ -420,7 +420,7 @@ public static FlowController createClusteredInstance(
private FlowController(
final FlowFileEventRepository flowFileEventRepo,
- final NiFiProperties properties,
+ final NiFiProperties nifiProperties,
final Authorizer authorizer,
final AuditService auditService,
final StringEncryptor encryptor,
@@ -435,16 +435,16 @@ private FlowController(
maxEventDrivenThreads = new AtomicInteger(5);
this.encryptor = encryptor;
- this.properties = properties;
+ this.nifiProperties = nifiProperties;
this.heartbeatMonitor = heartbeatMonitor;
- sslContext = SslContextFactory.createSslContext(properties, false);
+ sslContext = SslContextFactory.createSslContext(nifiProperties, false);
extensionManager = new ExtensionManager();
this.clusterCoordinator = clusterCoordinator;
timerDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxTimerDrivenThreads.get(), "Timer-Driven Process"));
eventDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxEventDrivenThreads.get(), "Event-Driven Process"));
- final FlowFileRepository flowFileRepo = createFlowFileRepository(properties, resourceClaimManager);
+ final FlowFileRepository flowFileRepo = createFlowFileRepository(nifiProperties, resourceClaimManager);
flowFileRepository = flowFileRepo;
flowFileEventRepository = flowFileEventRepo;
counterRepositoryRef = new AtomicReference<>(new StandardCounterRepository());
@@ -453,25 +453,25 @@ private FlowController(
this.variableRegistry = variableRegistry == null ? VariableRegistry.EMPTY_REGISTRY : variableRegistry;
try {
- this.provenanceRepository = createProvenanceRepository(properties);
+ this.provenanceRepository = createProvenanceRepository(nifiProperties);
this.provenanceRepository.initialize(createEventReporter(bulletinRepository), authorizer, this);
} catch (final Exception e) {
throw new RuntimeException("Unable to create Provenance Repository", e);
}
try {
- this.contentRepository = createContentRepository(properties);
+ this.contentRepository = createContentRepository(nifiProperties);
} catch (final Exception e) {
throw new RuntimeException("Unable to create Content Repository", e);
}
try {
- this.stateManagerProvider = StandardStateManagerProvider.create(properties, this.variableRegistry);
+ this.stateManagerProvider = StandardStateManagerProvider.create(nifiProperties, this.variableRegistry);
} catch (final IOException e) {
throw new RuntimeException(e);
}
- processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider, this.variableRegistry);
+ processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider, this.variableRegistry, this.nifiProperties);
eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
@@ -479,7 +479,7 @@ private FlowController(
eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, this.variableRegistry));
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
- final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
+ final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry, this.nifiProperties);
processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenAgent);
processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent);
processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, quartzSchedulingAgent);
@@ -490,7 +490,7 @@ private FlowController(
this.authorizer = authorizer;
this.auditService = auditService;
- final String gracefulShutdownSecondsVal = properties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
+ final String gracefulShutdownSecondsVal = nifiProperties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
long shutdownSecs;
try {
shutdownSecs = Long.parseLong(gracefulShutdownSecondsVal);
@@ -502,27 +502,26 @@ private FlowController(
}
gracefulShutdownSeconds = shutdownSecs;
- remoteInputSocketPort = properties.getRemoteInputPort();
- remoteInputHttpPort = properties.getRemoteInputHttpPort();
- isSiteToSiteSecure = properties.isSiteToSiteSecure();
+ remoteInputSocketPort = nifiProperties.getRemoteInputPort();
+ remoteInputHttpPort = nifiProperties.getRemoteInputHttpPort();
+ isSiteToSiteSecure = nifiProperties.isSiteToSiteSecure();
if (isSiteToSiteSecure && sslContext == null && remoteInputSocketPort != null) {
throw new IllegalStateException("NiFi Configured to allow Secure Site-to-Site communications but the Keystore/Truststore properties are not configured");
}
this.configuredForClustering = configuredForClustering;
- this.heartbeatDelaySeconds = (int) FormatUtils.getTimeDuration(properties.getNodeHeartbeatInterval(), TimeUnit.SECONDS);
+ this.heartbeatDelaySeconds = (int) FormatUtils.getTimeDuration(nifiProperties.getNodeHeartbeatInterval(), TimeUnit.SECONDS);
this.snippetManager = new SnippetManager();
-
final ProcessGroup rootGroup = new StandardProcessGroup(ComponentIdGenerator.generateId().toString(), this, processScheduler,
- properties, encryptor, this, this.variableRegistry);
+ nifiProperties, encryptor, this, this.variableRegistry);
rootGroup.setName(DEFAULT_ROOT_GROUP_NAME);
rootGroupRef.set(rootGroup);
instanceId = ComponentIdGenerator.generateId().toString();
- controllerServiceProvider = new StandardControllerServiceProvider(this, processScheduler, bulletinRepository, stateManagerProvider, this.variableRegistry);
+ controllerServiceProvider = new StandardControllerServiceProvider(this, processScheduler, bulletinRepository, stateManagerProvider, this.variableRegistry, this.nifiProperties);
if (remoteInputSocketPort == null) {
LOG.info("Not enabling RAW Socket Site-to-Site functionality because nifi.remote.input.socket.port is not set");
@@ -534,13 +533,13 @@ private FlowController(
RemoteResourceManager.setServerProtocolImplementation(SocketFlowFileServerProtocol.RESOURCE_NAME, SocketFlowFileServerProtocol.class);
final NodeInformant nodeInformant = configuredForClustering ? new ClusterCoordinatorNodeInformant(clusterCoordinator) : null;
- externalSiteListeners.add(new SocketRemoteSiteListener(remoteInputSocketPort, isSiteToSiteSecure ? sslContext : null, nodeInformant));
+ externalSiteListeners.add(new SocketRemoteSiteListener(remoteInputSocketPort, isSiteToSiteSecure ? sslContext : null, nifiProperties, nodeInformant));
}
if (remoteInputHttpPort == null) {
LOG.info("Not enabling HTTP(S) Site-to-Site functionality because the '" + NiFiProperties.SITE_TO_SITE_HTTP_ENABLED + "' property is not true");
} else {
- externalSiteListeners.add(HttpRemoteSiteListener.getInstance());
+ externalSiteListeners.add(HttpRemoteSiteListener.getInstance(nifiProperties));
}
for (final RemoteSiteListener listener : externalSiteListeners) {
@@ -548,7 +547,7 @@ private FlowController(
}
// Determine frequency for obtaining component status snapshots
- final String snapshotFrequency = properties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
+ final String snapshotFrequency = nifiProperties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
long snapshotMillis;
try {
snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
@@ -557,9 +556,9 @@ private FlowController(
}
// Initialize the Embedded ZooKeeper server, if applicable
- if (properties.isStartEmbeddedZooKeeper() && configuredForClustering) {
+ if (nifiProperties.isStartEmbeddedZooKeeper() && configuredForClustering) {
try {
- zooKeeperStateServer = ZooKeeperStateServer.create(properties);
+ zooKeeperStateServer = ZooKeeperStateServer.create(nifiProperties);
zooKeeperStateServer.start();
} catch (final IOException | ConfigException e) {
throw new IllegalStateException("Unable to initailize Flow because NiFi was configured to start an Embedded Zookeeper server but failed to do so", e);
@@ -580,8 +579,8 @@ public void run() {
heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false));
if (configuredForClustering) {
- leaderElectionManager = new CuratorLeaderElectionManager(4, properties);
- heartbeater = new ClusterProtocolHeartbeater(protocolSender, properties);
+ leaderElectionManager = new CuratorLeaderElectionManager(4, nifiProperties);
+ heartbeater = new ClusterProtocolHeartbeater(protocolSender, nifiProperties);
// Check if there is already a cluster coordinator elected. If not, go ahead
// and register for coordinator role. If there is already one elected, do not register until
@@ -624,7 +623,7 @@ private static FlowFileRepository createFlowFileRepository(final NiFiProperties
}
try {
- final FlowFileRepository created = NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileRepository.class);
+ final FlowFileRepository created = NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileRepository.class, properties);
synchronized (created) {
created.initialize(contentClaimManager);
}
@@ -641,7 +640,7 @@ private static FlowFileSwapManager createSwapManager(final NiFiProperties proper
}
try {
- return NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileSwapManager.class);
+ return NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileSwapManager.class, properties);
} catch (final Exception e) {
throw new RuntimeException(e);
}
@@ -820,7 +819,7 @@ private ContentRepository createContentRepository(final NiFiProperties propertie
}
try {
- final ContentRepository contentRepo = NarThreadContextClassLoader.createInstance(implementationClassName, ContentRepository.class);
+ final ContentRepository contentRepo = NarThreadContextClassLoader.createInstance(implementationClassName, ContentRepository.class, properties);
synchronized (contentRepo) {
contentRepo.initialize(resourceClaimManager);
}
@@ -838,21 +837,21 @@ private ProvenanceRepository createProvenanceRepository(final NiFiProperties pro
}
try {
- return NarThreadContextClassLoader.createInstance(implementationClassName, ProvenanceRepository.class);
+ return NarThreadContextClassLoader.createInstance(implementationClassName, ProvenanceRepository.class, properties);
} catch (final Exception e) {
throw new RuntimeException(e);
}
}
private ComponentStatusRepository createComponentStatusRepository() {
- final String implementationClassName = properties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
+ final String implementationClassName = nifiProperties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
if (implementationClassName == null) {
throw new RuntimeException("Cannot create Component Status Repository because the NiFi Properties is missing the following property: "
+ NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
}
try {
- return NarThreadContextClassLoader.createInstance(implementationClassName, ComponentStatusRepository.class);
+ return NarThreadContextClassLoader.createInstance(implementationClassName, ComponentStatusRepository.class, nifiProperties);
} catch (final Exception e) {
throw new RuntimeException(e);
}
@@ -883,7 +882,7 @@ public Connection createConnection(final String id, final String name, final Con
}
// Create and initialize a FlowFileSwapManager for this connection
- final FlowFileSwapManager swapManager = createSwapManager(properties);
+ final FlowFileSwapManager swapManager = createSwapManager(nifiProperties);
final EventReporter eventReporter = createEventReporter(getBulletinRepository());
try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
@@ -984,7 +983,7 @@ public Port createLocalOutputPort(String id, String name) {
* @throws NullPointerException if the argument is null
*/
public ProcessGroup createProcessGroup(final String id) {
- return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, properties, encryptor, this, variableRegistry);
+ return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, nifiProperties, encryptor, this, variableRegistry);
}
/**
@@ -1040,11 +1039,11 @@ public ProcessorNode createProcessor(final String type, String id, final boolean
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
final ProcessorNode procNode;
if (creationSuccessful) {
- procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider);
+ procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, nifiProperties);
} else {
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
final String componentType = "(Missing) " + simpleClassName;
- procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type);
+ procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type, nifiProperties);
}
final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
@@ -1087,7 +1086,7 @@ private Processor instantiateProcessor(final String type, final String identifie
final Class extends Processor> processorClass = rawClass.asSubclass(Processor.class);
processor = processorClass.newInstance();
final ComponentLog componentLogger = new SimpleProcessLogger(identifier, processor);
- final ProcessorInitializationContext ctx = new StandardProcessorInitializationContext(identifier, componentLogger, this, this);
+ final ProcessorInitializationContext ctx = new StandardProcessorInitializationContext(identifier, componentLogger, this, this, nifiProperties);
processor.initialize(ctx);
LogRepositoryFactory.getRepository(identifier).setLogger(componentLogger);
@@ -1191,7 +1190,7 @@ public Port createRemoteOutputPort(String id, String name) {
* @throws IllegalArgumentException if uri is not a valid URI.
*/
public RemoteProcessGroup createRemoteProcessGroup(final String id, final String uri) {
- return new StandardRemoteProcessGroup(requireNonNull(id).intern(), requireNonNull(uri).intern(), null, this, sslContext);
+ return new StandardRemoteProcessGroup(requireNonNull(id).intern(), requireNonNull(uri).intern(), null, this, sslContext, nifiProperties);
}
public ProcessGroup getRootGroup() {
@@ -2845,7 +2844,7 @@ public ReportingTaskNode createReportingTask(final String type, final String id,
if (firstTimeAdded) {
final ComponentLog componentLog = new SimpleProcessLogger(id, taskNode.getReportingTask());
final ReportingInitializationContext config = new StandardReportingInitializationContext(id, taskNode.getName(),
- SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this);
+ SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this, nifiProperties);
try {
task.initialize(config);
@@ -3305,7 +3304,6 @@ public boolean isConfiguredForClustering() {
return configuredForClustering;
}
-
private void registerForClusterCoordinator() {
leaderElectionManager.register(ClusterRoles.CLUSTER_COORDINATOR, new LeaderElectionStateChangeListener() {
@Override
@@ -3320,7 +3318,6 @@ public synchronized void onLeaderRelinquish() {
// call start() when we become the leader, and this will ensure that initialization is handled. The heartbeat monitor
// then will check the zookeeper znode to check if it is the cluster coordinator before kicking any nodes out of the
// cluster.
-
if (clusterCoordinator != null) {
clusterCoordinator.removeRole(ClusterRoles.CLUSTER_COORDINATOR);
}
@@ -3357,7 +3354,8 @@ public void onLeaderRelinquish() {
* either connected or trying to connect to the cluster.
*
* @param clustered true if clustered
- * @param clusterInstanceId if clustered is true, indicates the InstanceID of the Cluster Manager
+ * @param clusterInstanceId if clustered is true, indicates the InstanceID
+ * of the Cluster Manager
*/
public void setClustered(final boolean clustered, final String clusterInstanceId) {
writeLock.lock();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
index 091e59ca096c..6ad997677877 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
@@ -135,6 +135,8 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
*/
private NodeIdentifier nodeId;
+ private final NiFiProperties nifiProperties;
+
// guardedBy rwLock
private boolean firstControllerInitialization = true;
@@ -142,44 +144,45 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
private static final Logger logger = LoggerFactory.getLogger(StandardFlowService.class);
public static StandardFlowService createStandaloneInstance(
- final FlowController controller,
- final NiFiProperties properties,
- final StringEncryptor encryptor,
- final RevisionManager revisionManager,
- final Authorizer authorizer) throws IOException {
+ final FlowController controller,
+ final NiFiProperties nifiProperties,
+ final StringEncryptor encryptor,
+ final RevisionManager revisionManager,
+ final Authorizer authorizer) throws IOException {
- return new StandardFlowService(controller, properties, null, encryptor, false, null, revisionManager, authorizer);
+ return new StandardFlowService(controller, nifiProperties, null, encryptor, false, null, revisionManager, authorizer);
}
public static StandardFlowService createClusteredInstance(
- final FlowController controller,
- final NiFiProperties properties,
- final NodeProtocolSenderListener senderListener,
- final ClusterCoordinator coordinator,
- final StringEncryptor encryptor,
- final RevisionManager revisionManager,
- final Authorizer authorizer) throws IOException {
-
- return new StandardFlowService(controller, properties, senderListener, encryptor, true, coordinator, revisionManager, authorizer);
+ final FlowController controller,
+ final NiFiProperties nifiProperties,
+ final NodeProtocolSenderListener senderListener,
+ final ClusterCoordinator coordinator,
+ final StringEncryptor encryptor,
+ final RevisionManager revisionManager,
+ final Authorizer authorizer) throws IOException {
+
+ return new StandardFlowService(controller, nifiProperties, senderListener, encryptor, true, coordinator, revisionManager, authorizer);
}
private StandardFlowService(
- final FlowController controller,
- final NiFiProperties properties,
- final NodeProtocolSenderListener senderListener,
- final StringEncryptor encryptor,
- final boolean configuredForClustering,
- final ClusterCoordinator clusterCoordinator,
- final RevisionManager revisionManager,
- final Authorizer authorizer) throws IOException {
-
+ final FlowController controller,
+ final NiFiProperties nifiProperties,
+ final NodeProtocolSenderListener senderListener,
+ final StringEncryptor encryptor,
+ final boolean configuredForClustering,
+ final ClusterCoordinator clusterCoordinator,
+ final RevisionManager revisionManager,
+ final Authorizer authorizer) throws IOException {
+
+ this.nifiProperties = nifiProperties;
this.controller = controller;
- flowXml = Paths.get(properties.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE));
+ flowXml = Paths.get(nifiProperties.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE));
- gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(properties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS);
- autoResumeState = properties.getAutoResumeState();
+ gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(nifiProperties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS);
+ autoResumeState = nifiProperties.getAutoResumeState();
- dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor);
+ dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor, nifiProperties);
this.clusterCoordinator = clusterCoordinator;
if (clusterCoordinator != null) {
clusterCoordinator.setFlowService(this);
@@ -193,8 +196,8 @@ private StandardFlowService(
this.senderListener = senderListener;
senderListener.addHandler(this);
- final InetSocketAddress nodeApiAddress = properties.getNodeApiAddress();
- final InetSocketAddress nodeSocketAddress = properties.getClusterNodeProtocolAddress();
+ final InetSocketAddress nodeApiAddress = nifiProperties.getNodeApiAddress();
+ final InetSocketAddress nodeSocketAddress = nifiProperties.getClusterNodeProtocolAddress();
String nodeUuid = null;
final StateManager stateManager = controller.getStateManagerProvider().getStateManager(CLUSTER_NODE_CONFIG);
@@ -208,10 +211,10 @@ private StandardFlowService(
// use a random UUID as the proposed node identifier
this.nodeId = new NodeIdentifier(nodeUuid,
- nodeApiAddress.getHostName(), nodeApiAddress.getPort(),
- nodeSocketAddress.getHostName(), nodeSocketAddress.getPort(),
- properties.getRemoteInputHost(), properties.getRemoteInputPort(),
- properties.getRemoteInputHttpPort(), properties.isSiteToSiteSecure());
+ nodeApiAddress.getHostName(), nodeApiAddress.getPort(),
+ nodeSocketAddress.getHostName(), nodeSocketAddress.getPort(),
+ nifiProperties.getRemoteInputHost(), nifiProperties.getRemoteInputPort(),
+ nifiProperties.getRemoteInputHttpPort(), nifiProperties.isSiteToSiteSecure());
} else {
this.configuredForClustering = false;
@@ -244,7 +247,7 @@ public void saveFlowChanges(final OutputStream outStream) throws IOException {
public void overwriteFlow(final InputStream is) throws IOException {
writeLock.lock();
try (final OutputStream output = Files.newOutputStream(flowXml, StandardOpenOption.WRITE, StandardOpenOption.CREATE);
- final OutputStream gzipOut = new GZIPOutputStream(output);) {
+ final OutputStream gzipOut = new GZIPOutputStream(output);) {
FileUtils.copy(is, gzipOut);
} finally {
writeLock.unlock();
@@ -253,7 +256,7 @@ public void overwriteFlow(final InputStream is) throws IOException {
@Override
public void saveFlowChanges(final TimeUnit delayUnit, final long delay) {
- final boolean archiveEnabled = NiFiProperties.getInstance().isFlowConfigurationArchiveEnabled();
+ final boolean archiveEnabled = nifiProperties.isFlowConfigurationArchiveEnabled();
saveFlowChanges(delayUnit, delay, archiveEnabled);
}
@@ -584,12 +587,12 @@ private void handleReconnectionRequest(final ReconnectionRequestMessage request)
// reconnect
final ConnectionResponse connectionResponse = new ConnectionResponse(getNodeId(), request.getDataFlow(),
- request.getInstanceId(), request.getNodeConnectionStatuses(), request.getComponentRevisions());
+ request.getInstanceId(), request.getNodeConnectionStatuses(), request.getComponentRevisions());
loadFromConnectionResponse(connectionResponse);
clusterCoordinator.resetNodeStatuses(connectionResponse.getNodeConnectionStatuses().stream()
- .collect(Collectors.toMap(status -> status.getNodeIdentifier(), status -> status)));
+ .collect(Collectors.toMap(status -> status.getNodeIdentifier(), status -> status)));
controller.resumeHeartbeats(); // we are now connected, so resume sending heartbeats.
logger.info("Node reconnected.");
@@ -637,7 +640,7 @@ private void disconnect(final String explanation) {
// write lock must already be acquired
private void loadFromBytes(final DataFlow proposedFlow, final boolean allowEmptyFlow)
- throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
+ throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
logger.trace("Loading flow from bytes");
// resolve the given flow (null means load flow from disk)
@@ -695,16 +698,15 @@ private void loadFromBytes(final DataFlow proposedFlow, final boolean allowEmpty
}
/**
- * In NiFi 0.x, templates were stored in a templates directory as separate files. They are
- * now stored in the flow itself. If there already are templates in that directory, though,
- * we want to restore them.
+ * In NiFi 0.x, templates were stored in a templates directory as separate
+ * files. They are now stored in the flow itself. If there already are
+ * templates in that directory, though, we want to restore them.
*
* @return the templates found in the templates directory
* @throws IOException if unable to read from the file system
*/
public List loadTemplates() throws IOException {
- final NiFiProperties properties = NiFiProperties.getInstance();
- final Path templatePath = properties.getTemplateDirectory();
+ final Path templatePath = nifiProperties.getTemplateDirectory();
final File[] files = templatePath.toFile().listFiles(pathname -> {
final String lowerName = pathname.getName().toLowerCase();
@@ -718,7 +720,7 @@ public List loadTemplates() throws IOException {
final List templates = new ArrayList<>();
for (final File file : files) {
try (final FileInputStream fis = new FileInputStream(file);
- final BufferedInputStream bis = new BufferedInputStream(fis)) {
+ final BufferedInputStream bis = new BufferedInputStream(fis)) {
final TemplateDTO templateDto;
try {
@@ -820,7 +822,7 @@ private ConnectionResponse connect(final boolean retryOnCommsFailure, final bool
controller.getStateManagerProvider().getStateManager(CLUSTER_NODE_CONFIG).setState(map, Scope.LOCAL);
} catch (final IOException ioe) {
logger.warn("Received successful response from Cluster Manager but failed to persist state about the Node's Unique Identifier and the Node's Index. "
- + "This node may be assigned a different UUID when the node is restarted.", ioe);
+ + "This node may be assigned a different UUID when the node is restarted.", ioe);
}
return response;
@@ -834,7 +836,7 @@ private void loadFromConnectionResponse(final ConnectionResponse response) throw
writeLock.lock();
try {
clusterCoordinator.resetNodeStatuses(response.getNodeConnectionStatuses().stream()
- .collect(Collectors.toMap(status -> status.getNodeIdentifier(), status -> status)));
+ .collect(Collectors.toMap(status -> status.getNodeIdentifier(), status -> status)));
// get the dataflow from the response
final DataFlow dataFlow = response.getDataFlow();
@@ -869,7 +871,7 @@ private void loadFromConnectionResponse(final ConnectionResponse response) throw
throw new ConnectionException(CONNECTION_EXCEPTION_MSG_PREFIX + "local or cluster flow is malformed.", fse);
} catch (final FlowSynchronizationException fse) {
throw new FlowSynchronizationException(CONNECTION_EXCEPTION_MSG_PREFIX + "local flow controller partially updated. "
- + "Administrator should disconnect node and review flow for corruption.", fse);
+ + "Administrator should disconnect node and review flow for corruption.", fse);
} catch (final Exception ex) {
throw new ConnectionException("Failed to connect node to cluster due to: " + ex, ex);
} finally {
@@ -887,7 +889,7 @@ public void copyCurrentFlow(final OutputStream os) throws IOException {
}
try (final InputStream in = Files.newInputStream(flowXml, StandardOpenOption.READ);
- final InputStream gzipIn = new GZIPInputStream(in)) {
+ final InputStream gzipIn = new GZIPInputStream(in)) {
FileUtils.copy(gzipIn, os);
}
} finally {
@@ -895,7 +897,6 @@ public void copyCurrentFlow(final OutputStream os) throws IOException {
}
}
-
public void loadSnippets(final byte[] bytes) throws IOException {
if (bytes.length == 0) {
return;
@@ -909,7 +910,6 @@ public void loadSnippets(final byte[] bytes) throws IOException {
}
}
-
private class SaveReportingTask implements Runnable {
@Override
@@ -962,6 +962,7 @@ public void run() {
}
private class SaveHolder {
+
private final Calendar saveTime;
private final boolean shouldArchive;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
index d1822efebf12..6364d6c84859 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -121,10 +121,12 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
public static final URL FLOW_XSD_RESOURCE = StandardFlowSynchronizer.class.getResource("/FlowConfiguration.xsd");
private final StringEncryptor encryptor;
private final boolean autoResumeState;
+ private final NiFiProperties nifiProperties;
- public StandardFlowSynchronizer(final StringEncryptor encryptor) {
+ public StandardFlowSynchronizer(final StringEncryptor encryptor, final NiFiProperties nifiProperties) {
this.encryptor = encryptor;
- autoResumeState = NiFiProperties.getInstance().getAutoResumeState();
+ autoResumeState = nifiProperties.getAutoResumeState();
+ this.nifiProperties = nifiProperties;
}
public static boolean isEmpty(final DataFlow dataFlow) {
@@ -309,7 +311,7 @@ public void sync(final FlowController controller, final DataFlow proposedFlow, f
}
// get/create all the reporting task nodes and DTOs, but don't apply their scheduled state yet
- final Map reportingTaskNodesToDTOs = new HashMap<>();
+ final Map reportingTaskNodesToDTOs = new HashMap<>();
for (final Element taskElement : reportingTaskElements) {
final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(taskElement, encryptor);
final ReportingTaskNode reportingTask = getOrCreateReportingTask(controller, dto, initialized, existingFlowEmpty);
@@ -344,7 +346,7 @@ public void sync(final FlowController controller, final DataFlow proposedFlow, f
.collect(Collectors.toSet());
// clone the controller services and map the original id to the clone
- final Map controllerServiceMapping = new HashMap<>();
+ final Map controllerServiceMapping = new HashMap<>();
for (ControllerServiceNode controllerService : controllerServicesToClone) {
final ControllerServiceNode clone = ControllerServiceLoader.cloneControllerService(controller, controllerService);
controller.addRootControllerService(clone);
@@ -370,7 +372,7 @@ public void sync(final FlowController controller, final DataFlow proposedFlow, f
scaleRootGroup(rootGroup, encodingVersion);
// now that controller services are loaded and enabled we can apply the scheduled state to each reporting task
- for (Map.Entry entry : reportingTaskNodesToDTOs.entrySet()) {
+ for (Map.Entry entry : reportingTaskNodesToDTOs.entrySet()) {
applyReportingTaskScheduleState(controller, entry.getValue(), entry.getKey(), initialized, existingFlowEmpty);
}
}
@@ -403,15 +405,15 @@ public void sync(final FlowController controller, final DataFlow proposedFlow, f
}
}
- private void updateReportingTaskControllerServices(final Set reportingTasks, final Map controllerServiceMapping) {
+ private void updateReportingTaskControllerServices(final Set reportingTasks, final Map controllerServiceMapping) {
for (ReportingTaskNode reportingTask : reportingTasks) {
if (reportingTask.getProperties() != null) {
- final Set> propertyDescriptors = reportingTask.getProperties().entrySet().stream()
+ final Set> propertyDescriptors = reportingTask.getProperties().entrySet().stream()
.filter(e -> e.getKey().getControllerServiceDefinition() != null)
.filter(e -> controllerServiceMapping.containsKey(e.getValue()))
.collect(Collectors.toSet());
- for (Map.Entry propEntry : propertyDescriptors) {
+ for (Map.Entry propEntry : propertyDescriptors) {
final PropertyDescriptor propertyDescriptor = propEntry.getKey();
final ControllerServiceNode clone = controllerServiceMapping.get(propEntry.getValue());
reportingTask.setProperty(propertyDescriptor.getName(), clone.getIdentifier());
@@ -490,7 +492,7 @@ private static Document parseFlowBytes(final byte[] flow) throws FlowSerializati
}
private byte[] readFlowFromDisk() throws IOException {
- final Path flowPath = NiFiProperties.getInstance().getFlowConfigurationFile().toPath();
+ final Path flowPath = nifiProperties.getFlowConfigurationFile().toPath();
if (!Files.exists(flowPath) || Files.size(flowPath) == 0) {
return new byte[0];
}
@@ -544,7 +546,7 @@ private ReportingTaskNode getOrCreateReportingTask(final FlowController controll
final ComponentLog componentLog = new SimpleProcessLogger(dto.getId(), reportingTask.getReportingTask());
final ReportingInitializationContext config = new StandardReportingInitializationContext(dto.getId(), dto.getName(),
- SchedulingStrategy.valueOf(dto.getSchedulingStrategy()), dto.getSchedulingPeriod(), componentLog, controller);
+ SchedulingStrategy.valueOf(dto.getSchedulingStrategy()), dto.getSchedulingPeriod(), componentLog, controller, nifiProperties);
try {
reportingTask.getReportingTask().initialize(config);
@@ -560,7 +562,7 @@ private ReportingTaskNode getOrCreateReportingTask(final FlowController controll
}
private void applyReportingTaskScheduleState(final FlowController controller, final ReportingTaskDTO dto, final ReportingTaskNode reportingTask,
- final boolean controllerInitialized, final boolean existingFlowEmpty) {
+ final boolean controllerInitialized, final boolean existingFlowEmpty) {
if (!controllerInitialized || existingFlowEmpty) {
applyNewReportingTaskScheduleState(controller, dto, reportingTask);
} else {
@@ -636,7 +638,7 @@ private void applyExistingReportingTaskScheduleState(final FlowController contro
}
private ProcessGroup updateProcessGroup(final FlowController controller, final ProcessGroup parentGroup, final Element processGroupElement,
- final StringEncryptor encryptor, final FlowEncodingVersion encodingVersion) throws ProcessorInstantiationException {
+ final StringEncryptor encryptor, final FlowEncodingVersion encodingVersion) throws ProcessorInstantiationException {
// get the parent group ID
final String parentId = (parentGroup == null) ? null : parentGroup.getIdentifier();
@@ -792,10 +794,8 @@ private ProcessGroup updateProcessGroup(final FlowController controller, final P
if (inputPort.getScheduledState() != ScheduledState.RUNNING && inputPort.getScheduledState() != ScheduledState.STARTING) {
rpg.startTransmitting(inputPort);
}
- } else {
- if (inputPort.getScheduledState() != ScheduledState.STOPPED && inputPort.getScheduledState() != ScheduledState.STOPPING) {
- rpg.stopTransmitting(inputPort);
- }
+ } else if (inputPort.getScheduledState() != ScheduledState.STOPPED && inputPort.getScheduledState() != ScheduledState.STOPPING) {
+ rpg.stopTransmitting(inputPort);
}
}
@@ -813,15 +813,12 @@ private ProcessGroup updateProcessGroup(final FlowController controller, final P
if (outputPort.getScheduledState() != ScheduledState.RUNNING && outputPort.getScheduledState() != ScheduledState.STARTING) {
rpg.startTransmitting(outputPort);
}
- } else {
- if (outputPort.getScheduledState() != ScheduledState.STOPPED && outputPort.getScheduledState() != ScheduledState.STOPPING) {
- rpg.stopTransmitting(outputPort);
- }
+ } else if (outputPort.getScheduledState() != ScheduledState.STOPPED && outputPort.getScheduledState() != ScheduledState.STOPPING) {
+ rpg.stopTransmitting(outputPort);
}
}
}
-
// add labels
final List labelNodeList = getChildrenByTagName(processGroupElement, "label");
for (final Element labelElement : labelNodeList) {
@@ -969,7 +966,7 @@ private void updateProcessor(final ProcessorNode procNode, final ProcessorDTO pr
}
private ProcessGroup addProcessGroup(final FlowController controller, final ProcessGroup parentGroup, final Element processGroupElement,
- final StringEncryptor encryptor, final FlowEncodingVersion encodingVersion) throws ProcessorInstantiationException {
+ final StringEncryptor encryptor, final FlowEncodingVersion encodingVersion) throws ProcessorInstantiationException {
// get the parent group ID
final String parentId = (parentGroup == null) ? null : parentGroup.getIdentifier();
@@ -1285,12 +1282,15 @@ private ProcessGroup addProcessGroup(final FlowController controller, final Proc
}
/**
- * If both authorizers are external authorizers, or if the both are internal authorizers with equal fingerprints,
- * then an uniheritable result with no reason is returned to indicate nothing to do.
+ * If both authorizers are external authorizers, or if the both are internal
+ * authorizers with equal fingerprints, then an uniheritable result with no
+ * reason is returned to indicate nothing to do.
*
- * If both are internal authorizers and the current authorizer is empty, then an inheritable result is returned.
+ * If both are internal authorizers and the current authorizer is empty,
+ * then an inheritable result is returned.
*
- * All other cases return uninheritable with a reason which indicates to throw an exception.
+ * All other cases return uninheritable with a reason which indicates to
+ * throw an exception.
*
* @param existingFlow the existing DataFlow
* @param proposedFlow the proposed DataFlow
@@ -1336,13 +1336,15 @@ public AuthorizerInheritability checkAuthorizerInheritability(final DataFlow exi
}
/**
- * Returns true if the given controller can inherit the proposed flow without orphaning flow files.
+ * Returns true if the given controller can inherit the proposed flow
+ * without orphaning flow files.
*
* @param existingFlow flow
* @param controller the running controller
* @param proposedFlow the flow to inherit
*
- * @return null if the controller can inherit the specified flow, an explanation of why it cannot be inherited otherwise
+ * @return null if the controller can inherit the specified flow, an
+ * explanation of why it cannot be inherited otherwise
*
* @throws FingerprintException if flow fingerprints could not be generated
*/
@@ -1453,7 +1455,8 @@ private String formatFlowDiscrepancy(final String flowFingerprint, final int del
}
/**
- * Holder for the result of determining if a proposed Authorizer is inheritable.
+ * Holder for the result of determining if a proposed Authorizer is
+ * inheritable.
*/
private static final class AuthorizerInheritability {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index ab37aee0845f..5ff97ef1b481 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -127,22 +127,23 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
private final Requirement inputRequirement;
private final ProcessScheduler processScheduler;
private long runNanos = 0L;
+ private final NiFiProperties nifiProperties;
private SchedulingStrategy schedulingStrategy; // guarded by read/write lock
// ??????? NOT any more
public StandardProcessorNode(final Processor processor, final String uuid,
final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
- final ControllerServiceProvider controllerServiceProvider) {
+ final ControllerServiceProvider controllerServiceProvider, final NiFiProperties nifiProperties) {
this(processor, uuid, validationContextFactory, scheduler, controllerServiceProvider,
- processor.getClass().getSimpleName(), processor.getClass().getCanonicalName());
+ processor.getClass().getSimpleName(), processor.getClass().getCanonicalName(), nifiProperties);
}
public StandardProcessorNode(final Processor processor, final String uuid,
final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
final ControllerServiceProvider controllerServiceProvider,
- final String componentType, final String componentCanonicalClass) {
+ final String componentType, final String componentCanonicalClass, final NiFiProperties nifiProperties) {
super(processor, uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass);
@@ -166,6 +167,7 @@ public StandardProcessorNode(final Processor processor, final String uuid,
processScheduler = scheduler;
isolated = new AtomicBoolean(false);
penalizationPeriod = new AtomicReference<>(DEFAULT_PENALIZATION_PERIOD);
+ this.nifiProperties = nifiProperties;
final Class> procClass = processor.getClass();
triggerWhenEmpty = procClass.isAnnotationPresent(TriggerWhenEmpty.class);
@@ -1374,7 +1376,7 @@ public void run() {
*
*/
private void invokeTaskAsCancelableFuture(final SchedulingAgentCallback callback, final Callable task) {
- final String timeoutString = NiFiProperties.getInstance().getProperty(NiFiProperties.PROCESSOR_SCHEDULING_TIMEOUT);
+ final String timeoutString = nifiProperties.getProperty(NiFiProperties.PROCESSOR_SCHEDULING_TIMEOUT);
final long onScheduleTimeout = timeoutString == null ? 60000
: FormatUtils.getTimeDuration(timeoutString.trim(), TimeUnit.MILLISECONDS);
final Future> taskFuture = callback.invokeMonitoringTask(task);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java
index 02403184ec36..275fd3e39ece 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java
@@ -14,12 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.controller.cluster;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
-import java.util.Properties;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.CuratorFramework;
@@ -28,17 +26,20 @@
import org.apache.nifi.cluster.protocol.NodeProtocolSender;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
+import org.apache.nifi.util.NiFiProperties;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
- * Uses ZooKeeper in order to determine which node is the elected Cluster Coordinator and to indicate
- * that this node is part of the cluster. However, once the Cluster Coordinator is known, heartbeats are
- * sent directly to the Cluster Coordinator.
+ * Uses ZooKeeper in order to determine which node is the elected Cluster
+ * Coordinator and to indicate that this node is part of the cluster. However,
+ * once the Cluster Coordinator is known, heartbeats are sent directly to the
+ * Cluster Coordinator.
*/
public class ClusterProtocolHeartbeater implements Heartbeater {
+
private static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeater.class);
private final NodeProtocolSender protocolSender;
@@ -48,15 +49,14 @@ public class ClusterProtocolHeartbeater implements Heartbeater {
private final String coordinatorPath;
private volatile String coordinatorAddress;
-
- public ClusterProtocolHeartbeater(final NodeProtocolSender protocolSender, final Properties properties) {
+ public ClusterProtocolHeartbeater(final NodeProtocolSender protocolSender, final NiFiProperties nifiProperties) {
this.protocolSender = protocolSender;
final RetryPolicy retryPolicy = new RetryNTimes(10, 500);
- final ZooKeeperClientConfig zkConfig = ZooKeeperClientConfig.createConfig(properties);
+ final ZooKeeperClientConfig zkConfig = ZooKeeperClientConfig.createConfig(nifiProperties);
curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(),
- zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy);
+ zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy);
curatorClient.start();
nodesPathPrefix = zkConfig.resolvePath("cluster/nodes");
@@ -87,7 +87,6 @@ public void process(final WatchedEvent event) {
}
}
-
@Override
public synchronized void send(final HeartbeatMessage heartbeatMessage) throws IOException {
final String heartbeatAddress = getHeartbeatAddress();
@@ -107,7 +106,6 @@ public synchronized void send(final HeartbeatMessage heartbeatMessage) throws IO
}
}
-
@Override
public void close() throws IOException {
if (curatorClient != null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java
index 52cd7ec0a0ee..ffd4046f4090 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java
@@ -18,7 +18,6 @@
import java.util.ArrayList;
import java.util.List;
-import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import org.apache.commons.lang3.StringUtils;
@@ -70,8 +69,8 @@ public String resolvePath(final String path) {
return rootPath + "/" + path;
}
- public static ZooKeeperClientConfig createConfig(final Properties properties) {
- final String connectString = properties.getProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING);
+ public static ZooKeeperClientConfig createConfig(final NiFiProperties nifiProperties) {
+ final String connectString = nifiProperties.getProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING);
if (connectString == null || connectString.trim().isEmpty()) {
throw new IllegalStateException("The '" + NiFiProperties.ZOOKEEPER_CONNECT_STRING + "' property is not set in nifi.properties");
}
@@ -79,9 +78,9 @@ public static ZooKeeperClientConfig createConfig(final Properties properties) {
if (cleanedConnectString.isEmpty()) {
throw new IllegalStateException("The '" + NiFiProperties.ZOOKEEPER_CONNECT_STRING + "' property is set in nifi.properties but needs to be in pairs of host:port separated by commas");
}
- final long sessionTimeoutMs = getTimePeriod(properties, NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT);
- final long connectionTimeoutMs = getTimePeriod(properties, NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_CONNECT_TIMEOUT);
- final String rootPath = properties.getProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, NiFiProperties.DEFAULT_ZOOKEEPER_ROOT_NODE);
+ final long sessionTimeoutMs = getTimePeriod(nifiProperties, NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT);
+ final long connectionTimeoutMs = getTimePeriod(nifiProperties, NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_CONNECT_TIMEOUT);
+ final String rootPath = nifiProperties.getProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, NiFiProperties.DEFAULT_ZOOKEEPER_ROOT_NODE);
try {
PathUtils.validatePath(rootPath);
@@ -92,8 +91,8 @@ public static ZooKeeperClientConfig createConfig(final Properties properties) {
return new ZooKeeperClientConfig(cleanedConnectString, (int) sessionTimeoutMs, (int) connectionTimeoutMs, rootPath);
}
- private static int getTimePeriod(final Properties properties, final String propertyName, final String defaultValue) {
- final String timeout = properties.getProperty(propertyName, defaultValue);
+ private static int getTimePeriod(final NiFiProperties nifiProperties, final String propertyName, final String defaultValue) {
+ final String timeout = nifiProperties.getProperty(propertyName, defaultValue);
try {
return (int) FormatUtils.getTimeDuration(timeout, TimeUnit.MILLISECONDS);
} catch (final Exception e) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java
index 7bf749410d76..3ef2b8b7ebec 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java
@@ -14,12 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.controller.leader.election;
import java.util.HashMap;
import java.util.Map;
-import java.util.Properties;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.CuratorFramework;
@@ -37,6 +35,7 @@
import org.slf4j.LoggerFactory;
public class CuratorLeaderElectionManager implements LeaderElectionManager {
+
private static final Logger logger = LoggerFactory.getLogger(CuratorLeaderElectionManager.class);
private final FlowEngine leaderElectionMonitorEngine;
@@ -49,17 +48,11 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager {
private final Map leaderRoles = new HashMap<>();
private final Map registeredRoles = new HashMap<>();
-
- public CuratorLeaderElectionManager(final int threadPoolSize) {
- this(threadPoolSize, NiFiProperties.getInstance());
- }
-
- public CuratorLeaderElectionManager(final int threadPoolSize, final Properties properties) {
+ public CuratorLeaderElectionManager(final int threadPoolSize, final NiFiProperties properties) {
leaderElectionMonitorEngine = new FlowEngine(threadPoolSize, "Leader Election Notification", true);
zkConfig = ZooKeeperClientConfig.createConfig(properties);
}
-
@Override
public synchronized void start() {
if (!stopped) {
@@ -70,12 +63,12 @@ public synchronized void start() {
final RetryPolicy retryPolicy = new RetryForever(5000);
curatorClient = CuratorFrameworkFactory.builder()
- .connectString(zkConfig.getConnectString())
- .sessionTimeoutMs(zkConfig.getSessionTimeoutMillis())
- .connectionTimeoutMs(zkConfig.getConnectionTimeoutMillis())
- .retryPolicy(retryPolicy)
- .defaultData(new byte[0])
- .build();
+ .connectString(zkConfig.getConnectString())
+ .sessionTimeoutMs(zkConfig.getSessionTimeoutMillis())
+ .connectionTimeoutMs(zkConfig.getConnectionTimeoutMillis())
+ .retryPolicy(retryPolicy)
+ .defaultData(new byte[0])
+ .build();
curatorClient.start();
@@ -89,13 +82,11 @@ public synchronized void start() {
logger.info("{} started", this);
}
-
@Override
public synchronized void register(final String roleName) {
register(roleName, null);
}
-
@Override
public synchronized void register(final String roleName, final LeaderElectionStateChangeListener listener) {
logger.debug("{} Registering new Leader Selector for role {}", this, roleName);
@@ -168,13 +159,11 @@ public boolean isStopped() {
return stopped;
}
-
@Override
public String toString() {
return "CuratorLeaderElectionManager[stopped=" + isStopped() + "]";
}
-
@Override
public synchronized boolean isLeader(final String roleName) {
final LeaderRole role = leaderRoles.get(roleName);
@@ -185,8 +174,8 @@ public synchronized boolean isLeader(final String roleName) {
return role.isLeader();
}
-
private static class LeaderRole {
+
private final LeaderSelector leaderSelector;
private final ElectionListener electionListener;
@@ -204,8 +193,8 @@ public boolean isLeader() {
}
}
-
private class ElectionListener extends LeaderSelectorListenerAdapter implements LeaderSelectorListener {
+
private final String roleName;
private final LeaderElectionStateChangeListener listener;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java
index caf38a8a7d80..ae6354328d70 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller.reporting;
+import java.io.File;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@@ -26,6 +27,7 @@
import org.apache.nifi.reporting.ReportingInitializationContext;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
public class StandardReportingInitializationContext implements ReportingInitializationContext, ControllerServiceLookup {
@@ -35,15 +37,19 @@ public class StandardReportingInitializationContext implements ReportingInitiali
private final SchedulingStrategy schedulingStrategy;
private final ControllerServiceProvider serviceProvider;
private final ComponentLog logger;
+ private final NiFiProperties nifiProperties;
- public StandardReportingInitializationContext(final String id, final String name, final SchedulingStrategy schedulingStrategy,
- final String schedulingPeriod, final ComponentLog logger, final ControllerServiceProvider serviceProvider) {
+ public StandardReportingInitializationContext(
+ final String id, final String name, final SchedulingStrategy schedulingStrategy,
+ final String schedulingPeriod, final ComponentLog logger,
+ final ControllerServiceProvider serviceProvider, final NiFiProperties nifiProperties) {
this.id = id;
this.name = name;
this.schedulingPeriod = schedulingPeriod;
this.serviceProvider = serviceProvider;
this.schedulingStrategy = schedulingStrategy;
this.logger = logger;
+ this.nifiProperties = nifiProperties;
}
@Override
@@ -113,4 +119,19 @@ public String getControllerServiceName(final String serviceIdentifier) {
public ComponentLog getLogger() {
return logger;
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return nifiProperties.getKerberosServicePrincipal();
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return new File(nifiProperties.getKerberosKeytabLocation());
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return nifiProperties.getKerberosConfigurationFile();
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
index 8cf2401a5748..4e70e7bd1ef2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
@@ -126,10 +126,26 @@ public class FileSystemRepository implements ContentRepository {
// guarded by synchronizing on this
private final AtomicLong oldestArchiveDate = new AtomicLong(0L);
- public FileSystemRepository() throws IOException {
- final NiFiProperties properties = NiFiProperties.getInstance();
+ private final NiFiProperties nifiProperties;
+
+ /**
+ * Default no args constructor for service loading only
+ */
+ public FileSystemRepository() {
+ containers = null;
+ containerNames = null;
+ index = null;
+ archiveData = false;
+ maxArchiveMillis = 0;
+ alwaysSync = false;
+ containerCleanupExecutor = null;
+ nifiProperties = null;
+ }
+
+ public FileSystemRepository(final NiFiProperties nifiProperties) throws IOException {
+ this.nifiProperties = nifiProperties;
// determine the file repository paths and ensure they exist
- final Map fileRespositoryPaths = properties.getContentRepositoryPaths();
+ final Map fileRespositoryPaths = nifiProperties.getContentRepositoryPaths();
for (final Path path : fileRespositoryPaths.values()) {
Files.createDirectories(path);
}
@@ -139,21 +155,21 @@ public FileSystemRepository() throws IOException {
index = new AtomicLong(0L);
for (final String containerName : containerNames) {
- reclaimable.put(containerName, new LinkedBlockingQueue(10000));
- archivedFiles.put(containerName, new LinkedBlockingQueue(100000));
+ reclaimable.put(containerName, new LinkedBlockingQueue<>(10000));
+ archivedFiles.put(containerName, new LinkedBlockingQueue<>(100000));
}
- final String enableArchiving = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_ENABLED);
- final String maxArchiveRetentionPeriod = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_MAX_RETENTION_PERIOD);
- final String maxArchiveSize = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE);
- final String archiveBackPressureSize = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_BACK_PRESSURE_PERCENTAGE);
+ final String enableArchiving = nifiProperties.getProperty(NiFiProperties.CONTENT_ARCHIVE_ENABLED);
+ final String maxArchiveRetentionPeriod = nifiProperties.getProperty(NiFiProperties.CONTENT_ARCHIVE_MAX_RETENTION_PERIOD);
+ final String maxArchiveSize = nifiProperties.getProperty(NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE);
+ final String archiveBackPressureSize = nifiProperties.getProperty(NiFiProperties.CONTENT_ARCHIVE_BACK_PRESSURE_PERCENTAGE);
if ("true".equalsIgnoreCase(enableArchiving)) {
archiveData = true;
if (maxArchiveSize == null) {
throw new RuntimeException("No value specified for property '"
- + NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE + "' but archiving is enabled. You must configure the max disk usage in order to enable archiving.");
+ + NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE + "' but archiving is enabled. You must configure the max disk usage in order to enable archiving.");
}
if (!MAX_ARCHIVE_SIZE_PATTERN.matcher(maxArchiveSize.trim()).matches()) {
@@ -187,7 +203,7 @@ public FileSystemRepository() throws IOException {
final long maxArchiveBytes = (long) (capacity * (1D - (maxArchiveRatio - 0.02)));
minUsableContainerBytesForArchive.put(container.getKey(), Long.valueOf(maxArchiveBytes));
LOG.info("Maximum Threshold for Container {} set to {} bytes; if volume exceeds this size, archived data will be deleted until it no longer exceeds this size",
- containerName, maxArchiveBytes);
+ containerName, maxArchiveBytes);
final long backPressureBytes = (long) (Files.getFileStore(container.getValue()).getTotalSpace() * archiveBackPressureRatio);
final ContainerState containerState = new ContainerState(containerName, true, backPressureBytes, capacity);
@@ -205,7 +221,7 @@ public FileSystemRepository() throws IOException {
maxArchiveMillis = StringUtils.isEmpty(maxArchiveRetentionPeriod) ? Long.MAX_VALUE : FormatUtils.getTimeDuration(maxArchiveRetentionPeriod, TimeUnit.MILLISECONDS);
}
- this.alwaysSync = Boolean.parseBoolean(properties.getProperty("nifi.content.repository.always.sync"));
+ this.alwaysSync = Boolean.parseBoolean(nifiProperties.getProperty("nifi.content.repository.always.sync"));
LOG.info("Initializing FileSystemRepository with 'Always Sync' set to {}", alwaysSync);
initializeRepository();
@@ -216,16 +232,14 @@ public FileSystemRepository() throws IOException {
public void initialize(final ResourceClaimManager claimManager) {
this.resourceClaimManager = claimManager;
- final NiFiProperties properties = NiFiProperties.getInstance();
-
- final Map fileRespositoryPaths = properties.getContentRepositoryPaths();
+ final Map fileRespositoryPaths = nifiProperties.getContentRepositoryPaths();
executor.scheduleWithFixedDelay(new BinDestructableClaims(), 1, 1, TimeUnit.SECONDS);
for (int i = 0; i < fileRespositoryPaths.size(); i++) {
executor.scheduleWithFixedDelay(new ArchiveOrDestroyDestructableClaims(), 1, 1, TimeUnit.SECONDS);
}
- final long cleanupMillis = this.determineCleanupInterval(properties);
+ final long cleanupMillis = this.determineCleanupInterval(nifiProperties);
for (final Map.Entry containerEntry : containers.entrySet()) {
final String containerName = containerEntry.getKey();
@@ -562,7 +576,6 @@ protected int incrementClaimantCount(final ResourceClaim resourceClaim, final bo
return resourceClaimManager.incrementClaimantCount(resourceClaim, newClaim);
}
-
@Override
public int getClaimantCount(final ContentClaim claim) {
if (claim == null) {
@@ -619,14 +632,13 @@ private boolean remove(final ResourceClaim claim) {
final File file = path.toFile();
if (!file.delete() && file.exists()) {
- LOG.warn("Unable to delete {} at path {}", new Object[] {claim, path});
+ LOG.warn("Unable to delete {} at path {}", new Object[]{claim, path});
return false;
}
return true;
}
-
@Override
public ContentClaim clone(final ContentClaim original, final boolean lossTolerant) throws IOException {
if (original == null) {
@@ -635,7 +647,7 @@ public ContentClaim clone(final ContentClaim original, final boolean lossToleran
final ContentClaim newClaim = create(lossTolerant);
try (final InputStream in = read(original);
- final OutputStream out = write(newClaim)) {
+ final OutputStream out = write(newClaim)) {
StreamUtils.copy(in, out);
} catch (final IOException ioe) {
decrementClaimantCount(newClaim);
@@ -700,7 +712,7 @@ public long exportTo(final ContentClaim claim, final Path destination, final boo
}
try (final InputStream in = read(claim);
- final FileOutputStream fos = new FileOutputStream(destination.toFile(), append)) {
+ final FileOutputStream fos = new FileOutputStream(destination.toFile(), append)) {
final long copied = StreamUtils.copy(in, fos);
if (alwaysSync) {
fos.getFD().sync();
@@ -729,7 +741,7 @@ public long exportTo(final ContentClaim claim, final Path destination, final boo
}
try (final InputStream in = read(claim);
- final FileOutputStream fos = new FileOutputStream(destination.toFile(), append)) {
+ final FileOutputStream fos = new FileOutputStream(destination.toFile(), append)) {
if (offset > 0) {
StreamUtils.skip(in, offset);
}
@@ -801,7 +813,7 @@ public InputStream read(final ContentClaim claim) throws IOException {
if (claim.getOffset() > 0L) {
try {
StreamUtils.skip(fis, claim.getOffset());
- } catch(IOException ioe) {
+ } catch (IOException ioe) {
IOUtils.closeQuietly(fis);
throw ioe;
}
@@ -821,7 +833,6 @@ public OutputStream write(final ContentClaim claim) throws IOException {
return write(claim, false);
}
-
private OutputStream write(final ContentClaim claim, final boolean append) throws IOException {
if (claim == null) {
throw new NullPointerException("ContentClaim cannot be null");
@@ -973,7 +984,6 @@ public synchronized void close() throws IOException {
return out;
}
-
@Override
public void purge() {
// delete all content from repositories
@@ -1035,7 +1045,7 @@ public void run() {
break;
} else {
LOG.warn("Failed to clean up {} because old claims aren't being cleaned up fast enough. "
- + "This Content Claim will remain in the Content Repository until NiFi is restarted, at which point it will be cleaned up", claim);
+ + "This Content Claim will remain in the Content Repository until NiFi is restarted, at which point it will be cleaned up", claim);
}
}
} catch (final InterruptedException ie) {
@@ -1210,10 +1220,10 @@ private long destroyExpiredArchives(final String containerName, final Path conta
if (archiveExpirationLog.isDebugEnabled()) {
if (toFree < 0) {
archiveExpirationLog.debug("Currently {} bytes free for Container {}; requirement is {} byte free, so no need to free space until an additional {} bytes are used",
- usableSpace, containerName, minRequiredSpace, Math.abs(toFree));
+ usableSpace, containerName, minRequiredSpace, Math.abs(toFree));
} else {
archiveExpirationLog.debug("Currently {} bytes free for Container {}; requirement is {} byte free, so need to free {} bytes",
- usableSpace, containerName, minRequiredSpace, toFree);
+ usableSpace, containerName, minRequiredSpace, toFree);
}
}
@@ -1256,10 +1266,10 @@ private long destroyExpiredArchives(final String containerName, final Path conta
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
if (deleteCount > 0) {
LOG.info("Deleted {} files from archive for Container {}; oldest Archive Date is now {}; container cleanup took {} millis",
- deleteCount, containerName, new Date(oldestArchiveDate), millis);
+ deleteCount, containerName, new Date(oldestArchiveDate), millis);
} else {
LOG.debug("Deleted {} files from archive for Container {}; oldest Archive Date is now {}; container cleanup took {} millis",
- deleteCount, containerName, new Date(oldestArchiveDate), millis);
+ deleteCount, containerName, new Date(oldestArchiveDate), millis);
}
return oldestArchiveDate;
@@ -1297,7 +1307,7 @@ public FileVisitResult visitFile(final Path file, final BasicFileAttributes attr
Files.deleteIfExists(file);
containerState.decrementArchiveCount();
LOG.debug("Deleted archived ContentClaim with ID {} from Container {} because it was older than the configured max archival duration",
- file.toFile().getName(), containerName);
+ file.toFile().getName(), containerName);
} catch (final IOException ioe) {
LOG.warn("Failed to remove archived ContentClaim with ID {} from Container {} due to {}", file.toFile().getName(), containerName, ioe.toString());
if (LOG.isDebugEnabled()) {
@@ -1380,7 +1390,7 @@ public int compare(final ArchiveInfo o1, final ArchiveInfo o2) {
final long cleanupMillis = stopWatch.getElapsed(TimeUnit.MILLISECONDS) - deleteOldestMillis - sortRemainingMillis - deleteExpiredMillis;
LOG.debug("Oldest Archive Date for Container {} is {}; delete expired = {} ms, sort remaining = {} ms, delete oldest = {} ms, cleanup = {} ms",
- containerName, new Date(oldestContainerArchive), deleteExpiredMillis, sortRemainingMillis, deleteOldestMillis, cleanupMillis);
+ containerName, new Date(oldestContainerArchive), deleteExpiredMillis, sortRemainingMillis, deleteOldestMillis, cleanupMillis);
return oldestContainerArchive;
}
@@ -1420,10 +1430,8 @@ public void run() {
LOG.warn("", e);
}
}
- } else {
- if (remove(claim)) {
- successCount++;
- }
+ } else if (remove(claim)) {
+ successCount++;
}
}
@@ -1571,7 +1579,9 @@ public ContainerState(final String containerName, final boolean archiveEnabled,
}
/**
- * @return {@code true} if wait is required to create claims against this Container, based on whether or not the container has reached its back pressure threshold
+ * @return {@code true} if wait is required to create claims against
+ * this Container, based on whether or not the container has reached its
+ * back pressure threshold
*/
public boolean isWaitRequired() {
if (!archiveEnabled) {
@@ -1642,8 +1652,8 @@ public void decrementArchiveCount() {
}
}
-
private static class ClaimLengthPair {
+
private final ResourceClaim claim;
private final Long length;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
index 08b7e80a7f6e..95b503bcb27b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
@@ -56,24 +56,33 @@
/**
*
- * An in-memory implementation of the {@link ContentRepository} interface. This implementation stores FlowFile content in the Java heap and keeps track of the number of bytes used. If the number of
- * bytes used by FlowFile content exceeds some threshold (configured via the nifi.volatile.content.repository.max.size property in the NiFi properties with a default of 100 MB), one of
- * two situations will occur:
+ * An in-memory implementation of the {@link ContentRepository} interface. This
+ * implementation stores FlowFile content in the Java heap and keeps track of
+ * the number of bytes used. If the number of bytes used by FlowFile content
+ * exceeds some threshold (configured via the
+ * nifi.volatile.content.repository.max.size property in the NiFi
+ * properties with a default of 100 MB), one of two situations will occur:
*
*
*
- * - Backup Repository: If a Backup Repository has been specified (via the {@link #setBackupRepository(ContentRepository)} method), the content will be stored in the backup repository and all
- * access to the FlowFile content will automatically and transparently be proxied to the backup repository.
+ *
- Backup Repository: If a Backup Repository has been specified (via
+ * the {@link #setBackupRepository(ContentRepository)} method), the content will
+ * be stored in the backup repository and all access to the FlowFile content
+ * will automatically and transparently be proxied to the backup repository.
*
* -
- * Without Backup Repository: If no Backup Repository has been specified, when the threshold is exceeded, an IOException will be thrown.
+ * Without Backup Repository: If no Backup Repository has been specified,
+ * when the threshold is exceeded, an IOException will be thrown.
*
*
*
*
- * When a Content Claim is created via the {@link #create(boolean)} method, if the lossTolerant flag is set to false, the Backup Repository will be used to create the Content
- * Claim and any accesses to the ContentClaim will be proxied to the Backup Repository. If the Backup Repository has not been specified, attempting to create a non-loss-tolerant ContentClaim will
- * result in an {@link IllegalStateException} being thrown.
+ * When a Content Claim is created via the {@link #create(boolean)} method, if
+ * the lossTolerant flag is set to false, the Backup
+ * Repository will be used to create the Content Claim and any accesses to the
+ * ContentClaim will be proxied to the Backup Repository. If the Backup
+ * Repository has not been specified, attempting to create a non-loss-tolerant
+ * ContentClaim will result in an {@link IllegalStateException} being thrown.
*
*/
public class VolatileContentRepository implements ContentRepository {
@@ -98,13 +107,17 @@ public class VolatileContentRepository implements ContentRepository {
private ResourceClaimManager claimManager; // effectively final
+ /**
+ * Default no args constructor for service loading only
+ */
public VolatileContentRepository() {
- this(NiFiProperties.getInstance());
+ maxBytes = 0;
+ memoryManager = null;
}
- public VolatileContentRepository(final NiFiProperties properties) {
- final String maxSize = properties.getProperty(MAX_SIZE_PROPERTY);
- final String blockSizeVal = properties.getProperty(BLOCK_SIZE_PROPERTY);
+ public VolatileContentRepository(final NiFiProperties nifiProperties) {
+ final String maxSize = nifiProperties.getProperty(MAX_SIZE_PROPERTY);
+ final String blockSizeVal = nifiProperties.getProperty(BLOCK_SIZE_PROPERTY);
if (maxSize == null) {
maxBytes = (long) DataUnit.B.convert(100D, DataUnit.MB);
@@ -137,7 +150,8 @@ public void shutdown() {
}
/**
- * Specifies a Backup Repository where data should be written if this Repository fills up
+ * Specifies a Backup Repository where data should be written if this
+ * Repository fills up
*
* @param backup repo backup
*/
@@ -388,7 +402,7 @@ public long exportTo(final ContentClaim claim, final Path destination, final boo
final StandardOpenOption openOption = append ? StandardOpenOption.APPEND : StandardOpenOption.CREATE;
try (final InputStream in = read(claim);
- final OutputStream destinationStream = Files.newOutputStream(destination, openOption)) {
+ final OutputStream destinationStream = Files.newOutputStream(destination, openOption)) {
if (offset > 0) {
StreamUtils.skip(in, offset);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
index 2e5f00551a96..36d592c932a8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
@@ -63,12 +63,21 @@
*
*
*
- * We expose a property named nifi.flowfile.repository.always.sync that is a boolean value indicating whether or not to force WALI to sync with disk on each update. By default, the value
- * is false. This is needed only in situations in which power loss is expected and not mitigated by Uninterruptable Power Sources (UPS) or when running in an unstable Virtual Machine for
- * instance. Otherwise, we will flush the data that is written to the Operating System and the Operating System will be responsible to flush its buffers when appropriate. The Operating System can be
- * configured to hold only a certain buffer size or not to buffer at all, as well. When using a UPS, this is generally not an issue, as the machine is typically notified before dying, in which case
- * the Operating System will flush the data to disk. Additionally, most disks on enterprise servers also have battery backups that can power the disks long enough to flush their buffers. For this
- * reason, we choose instead to not sync to disk for every write but instead sync only when we checkpoint.
+ * We expose a property named nifi.flowfile.repository.always.sync
+ * that is a boolean value indicating whether or not to force WALI to sync with
+ * disk on each update. By default, the value is false. This is
+ * needed only in situations in which power loss is expected and not mitigated
+ * by Uninterruptable Power Sources (UPS) or when running in an unstable Virtual
+ * Machine for instance. Otherwise, we will flush the data that is written to
+ * the Operating System and the Operating System will be responsible to flush
+ * its buffers when appropriate. The Operating System can be configured to hold
+ * only a certain buffer size or not to buffer at all, as well. When using a
+ * UPS, this is generally not an issue, as the machine is typically notified
+ * before dying, in which case the Operating System will flush the data to disk.
+ * Additionally, most disks on enterprise servers also have battery backups that
+ * can power the disks long enough to flush their buffers. For this reason, we
+ * choose instead to not sync to disk for every write but instead sync only when
+ * we checkpoint.
*
*/
public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncListener {
@@ -112,15 +121,24 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
// on restart.
private final ConcurrentMap> claimsAwaitingDestruction = new ConcurrentHashMap<>();
+ /**
+ * default no args constructor for service loading only.
+ */
public WriteAheadFlowFileRepository() {
- final NiFiProperties properties = NiFiProperties.getInstance();
+ alwaysSync = false;
+ checkpointDelayMillis = 0l;
+ flowFileRepositoryPath = null;
+ numPartitions = 0;
+ checkpointExecutor = null;
+ }
- alwaysSync = Boolean.parseBoolean(properties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_ALWAYS_SYNC, "false"));
+ public WriteAheadFlowFileRepository(final NiFiProperties nifiProperties) {
+ alwaysSync = Boolean.parseBoolean(nifiProperties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_ALWAYS_SYNC, "false"));
// determine the database file path and ensure it exists
- flowFileRepositoryPath = properties.getFlowFileRepositoryPath();
- numPartitions = properties.getFlowFileRepositoryPartitions();
- checkpointDelayMillis = FormatUtils.getTimeDuration(properties.getFlowFileRepositoryCheckpointInterval(), TimeUnit.MILLISECONDS);
+ flowFileRepositoryPath = nifiProperties.getFlowFileRepositoryPath();
+ numPartitions = nifiProperties.getFlowFileRepositoryPartitions();
+ checkpointDelayMillis = FormatUtils.getTimeDuration(nifiProperties.getFlowFileRepositoryCheckpointInterval(), TimeUnit.MILLISECONDS);
checkpointExecutor = Executors.newSingleThreadScheduledExecutor();
}
@@ -253,7 +271,6 @@ private void updateRepository(final Collection records, final
}
}
-
@Override
public void onSync(final int partitionIndex) {
final BlockingQueue claimQueue = claimsAwaitingDestruction.get(Integer.valueOf(partitionIndex));
@@ -282,7 +299,9 @@ public void onGlobalSync() {
}
/**
- * Swaps the FlowFiles that live on the given Connection out to disk, using the specified Swap File and returns the number of FlowFiles that were persisted.
+ * Swaps the FlowFiles that live on the given Connection out to disk, using
+ * the specified Swap File and returns the number of FlowFiles that were
+ * persisted.
*
* @param queue queue to swap out
* @param swapLocation location to swap to
@@ -397,6 +416,7 @@ public int checkpoint() throws IOException {
}
private static class WriteAheadRecordSerde implements SerDe {
+
private static final int CURRENT_ENCODING_VERSION = 9;
public static final byte ACTION_CREATE = 0;
@@ -551,7 +571,7 @@ public RepositoryRecord deserializeEdit(final DataInputStream in, final Map 1) {
// read the lineage identifiers and lineage start date, which were added in version 2.
- if(version < 9){
+ if (version < 9) {
final int numLineageIds = in.readInt();
for (int i = 0; i < numLineageIds; i++) {
in.readUTF(); //skip identifiers
@@ -662,7 +682,7 @@ public StandardRepositoryRecord deserializeRecord(final DataInputStream in, fina
if (version > 1) {
// read the lineage identifiers and lineage start date, which were added in version 2.
- if(version < 9) {
+ if (version < 9) {
final int numLineageIds = in.readInt();
for (int i = 0; i < numLineageIds; i++) {
in.readUTF(); //skip identifiers
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index aadbff03de01..10a746ed8988 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -62,7 +62,8 @@
import org.slf4j.LoggerFactory;
/**
- * Responsible for scheduling Processors, Ports, and Funnels to run at regular intervals
+ * Responsible for scheduling Processors, Ports, and Funnels to run at regular
+ * intervals
*/
public final class StandardProcessScheduler implements ProcessScheduler {
@@ -84,14 +85,19 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private final StringEncryptor encryptor;
private final VariableRegistry variableRegistry;
- public StandardProcessScheduler(final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor,
- final StateManagerProvider stateManagerProvider, final VariableRegistry variableRegistry) {
+ public StandardProcessScheduler(
+ final ControllerServiceProvider controllerServiceProvider,
+ final StringEncryptor encryptor,
+ final StateManagerProvider stateManagerProvider,
+ final VariableRegistry variableRegistry,
+ final NiFiProperties nifiProperties
+ ) {
this.controllerServiceProvider = controllerServiceProvider;
this.encryptor = encryptor;
this.stateManagerProvider = stateManagerProvider;
this.variableRegistry = variableRegistry;
- administrativeYieldDuration = NiFiProperties.getInstance().getAdministrativeYieldDuration();
+ administrativeYieldDuration = nifiProperties.getAdministrativeYieldDuration();
administrativeYieldMillis = FormatUtils.getTimeDuration(administrativeYieldDuration, TimeUnit.MILLISECONDS);
frameworkTaskExecutor = new FlowEngine(4, "Framework Task Thread");
@@ -216,8 +222,8 @@ public void run() {
componentLog.error("Failed to invoke @OnEnabled method due to {}", cause);
LOG.error("Failed to invoke the On-Scheduled Lifecycle methods of {} due to {}; administratively yielding this "
- + "ReportingTask and will attempt to schedule it again after {}",
- new Object[] { reportingTask, e.toString(), administrativeYieldDuration }, e);
+ + "ReportingTask and will attempt to schedule it again after {}",
+ new Object[]{reportingTask, e.toString(), administrativeYieldDuration}, e);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, reportingTask, taskNode.getConfigurationContext());
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, reportingTask, taskNode.getConfigurationContext());
@@ -265,7 +271,7 @@ public void run() {
componentLog.error("Failed to invoke @OnUnscheduled method due to {}", cause);
LOG.error("Failed to invoke the @OnUnscheduled methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}",
- reportingTask, cause.toString(), administrativeYieldDuration);
+ reportingTask, cause.toString(), administrativeYieldDuration);
LOG.error("", cause);
try {
@@ -290,8 +296,9 @@ public void run() {
* Starts the given {@link Processor} by invoking its
* {@link ProcessorNode#start(ScheduledExecutorService, long, org.apache.nifi.processor.ProcessContext, Runnable)}
* .
+ *
* @see StandardProcessorNode#start(ScheduledExecutorService, long,
- * org.apache.nifi.processor.ProcessContext, Runnable).
+ * org.apache.nifi.processor.ProcessContext, Runnable).
*/
@Override
public synchronized void startProcessor(final ProcessorNode procNode) {
@@ -324,8 +331,9 @@ public void postMonitor() {
* Stops the given {@link Processor} by invoking its
* {@link ProcessorNode#stop(ScheduledExecutorService, org.apache.nifi.processor.ProcessContext, Callable)}
* .
+ *
* @see StandardProcessorNode#stop(ScheduledExecutorService,
- * org.apache.nifi.processor.ProcessContext, Callable)
+ * org.apache.nifi.processor.ProcessContext, Callable)
*/
@Override
public synchronized void stopProcessor(final ProcessorNode procNode) {
@@ -524,8 +532,7 @@ public boolean isScheduled(final Object scheduled) {
* no ScheduleState current is registered, one is created and registered
* atomically, and then that value is returned.
*
- * @param schedulable
- * schedulable
+ * @param schedulable schedulable
* @return scheduled state
*/
private ScheduleState getScheduleState(final Object schedulable) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
index f94befff966a..fcd901f4a4d3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
@@ -55,15 +55,20 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
private volatile String adminYieldDuration = "1 sec";
- public TimerDrivenSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor encryptor,
- final VariableRegistry variableRegistry) {
+ public TimerDrivenSchedulingAgent(
+ final FlowController flowController,
+ final FlowEngine flowEngine,
+ final ProcessContextFactory contextFactory,
+ final StringEncryptor encryptor,
+ final VariableRegistry variableRegistry,
+ final NiFiProperties nifiProperties) {
super(flowEngine);
this.flowController = flowController;
this.contextFactory = contextFactory;
this.encryptor = encryptor;
this.variableRegistry = variableRegistry;
- final String boredYieldDuration = NiFiProperties.getInstance().getBoredYieldDuration();
+ final String boredYieldDuration = nifiProperties.getBoredYieldDuration();
try {
noWorkYieldNanos = FormatUtils.getTimeDuration(boredYieldDuration, TimeUnit.NANOSECONDS);
} catch (final IllegalArgumentException e) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java
index 71cd793fcc75..59e668cb63fd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller.service;
+import java.io.File;
import java.util.Set;
import org.apache.nifi.components.state.StateManager;
@@ -23,6 +24,7 @@
import org.apache.nifi.controller.ControllerServiceInitializationContext;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.NiFiProperties;
public class StandardControllerServiceInitializationContext implements ControllerServiceInitializationContext, ControllerServiceLookup {
@@ -30,12 +32,17 @@ public class StandardControllerServiceInitializationContext implements Controlle
private final ControllerServiceProvider serviceProvider;
private final ComponentLog logger;
private final StateManager stateManager;
+ private final NiFiProperties nifiProperties;
- public StandardControllerServiceInitializationContext(final String identifier, final ComponentLog logger, final ControllerServiceProvider serviceProvider, final StateManager stateManager) {
+ public StandardControllerServiceInitializationContext(
+ final String identifier, final ComponentLog logger,
+ final ControllerServiceProvider serviceProvider, final StateManager stateManager,
+ final NiFiProperties nifiProperties) {
this.id = identifier;
this.logger = logger;
this.serviceProvider = serviceProvider;
this.stateManager = stateManager;
+ this.nifiProperties = nifiProperties;
}
@Override
@@ -87,4 +94,19 @@ public ComponentLog getLogger() {
public StateManager getStateManager() {
return stateManager;
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return nifiProperties.getKerberosServicePrincipal();
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return new File(nifiProperties.getKerberosKeytabLocation());
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return nifiProperties.getKerberosConfigurationFile();
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index 906526400f3c..9fd1ca246644 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -59,6 +59,7 @@
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -73,6 +74,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
private final StateManagerProvider stateManagerProvider;
private final VariableRegistry variableRegistry;
private final FlowController flowController;
+ private final NiFiProperties nifiProperties;
static {
// methods that are okay to be called when the service is disabled.
@@ -87,13 +89,14 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
public StandardControllerServiceProvider(final FlowController flowController, final ProcessScheduler scheduler, final BulletinRepository bulletinRepo,
- final StateManagerProvider stateManagerProvider, final VariableRegistry variableRegistry) {
+ final StateManagerProvider stateManagerProvider, final VariableRegistry variableRegistry, final NiFiProperties nifiProperties) {
this.flowController = flowController;
this.processScheduler = scheduler;
this.bulletinRepo = bulletinRepo;
this.stateManagerProvider = stateManagerProvider;
this.variableRegistry = variableRegistry;
+ this.nifiProperties = nifiProperties;
}
private Class>[] getInterfaces(final Class> cls) {
@@ -189,7 +192,7 @@ public Object invoke(final Object proxy, final Method method, final Object[] arg
logger.info("Created Controller Service of type {} with identifier {}", type, id);
final ComponentLog serviceLogger = new SimpleProcessLogger(id, originalService);
- originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this, getStateManager(id)));
+ originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this, getStateManager(id), nifiProperties));
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this, variableRegistry);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
index 828bdfec10d6..4c620d1c7f2c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java
@@ -40,10 +40,15 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
private volatile long lastCaptureTime = 0L;
+ /**
+ * Default no args constructor for service loading only
+ */
+ public VolatileComponentStatusRepository(){
+ captures = null;
+ }
- public VolatileComponentStatusRepository() {
- final NiFiProperties properties = NiFiProperties.getInstance();
- final int numDataPoints = properties.getIntegerProperty(NUM_DATA_POINTS_PROPERTY, DEFAULT_NUM_DATA_POINTS);
+ public VolatileComponentStatusRepository(final NiFiProperties nifiProperties) {
+ final int numDataPoints = nifiProperties.getIntegerProperty(NUM_DATA_POINTS_PROPERTY, DEFAULT_NUM_DATA_POINTS);
captures = new RingBuffer<>(numDataPoints);
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
index 5de1bebc40b6..15043c3544af 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
@@ -26,14 +26,16 @@
/**
*
- * An application specific string encryptor that collects configuration from the application properties, system properties, and/or system environment.
+ * An application specific string encryptor that collects configuration from the
+ * application properties, system properties, and/or system environment.
*
*
*
* Instance of this class are thread-safe
*
*
- * The encryption provider and algorithm is configured using the application properties:
+ * The encryption provider and algorithm is configured using the application
+ * properties:
*
* - nifi.sensitive.props.provider
* - nifi.sensitive.props.algorithm
@@ -71,18 +73,21 @@ private StringEncryptor(final String aglorithm, final String provider, final Str
}
/**
- * Creates an instance of the nifi sensitive property encryptor. Validates that the encryptor is actually working.
+ * Creates an instance of the nifi sensitive property encryptor. Validates
+ * that the encryptor is actually working.
*
+ * @param niFiProperties properties
* @return encryptor
- * @throws EncryptionException if any issues arise initializing or validating the encryptor
+ * @throws EncryptionException if any issues arise initializing or
+ * validating the encryptor
*/
- public static StringEncryptor createEncryptor() throws EncryptionException {
+ public static StringEncryptor createEncryptor(final NiFiProperties niFiProperties) throws EncryptionException {
Security.addProvider(new org.bouncycastle.jce.provider.BouncyCastleProvider());
- final String sensitivePropAlgorithmVal = NiFiProperties.getInstance().getProperty(NF_SENSITIVE_PROPS_ALGORITHM);
- final String sensitivePropProviderVal = NiFiProperties.getInstance().getProperty(NF_SENSITIVE_PROPS_PROVIDER);
- final String sensitivePropValueNifiPropVar = NiFiProperties.getInstance().getProperty(NF_SENSITIVE_PROPS_KEY, DEFAULT_SENSITIVE_PROPS_KEY);
+ final String sensitivePropAlgorithmVal = niFiProperties.getProperty(NF_SENSITIVE_PROPS_ALGORITHM);
+ final String sensitivePropProviderVal = niFiProperties.getProperty(NF_SENSITIVE_PROPS_PROVIDER);
+ final String sensitivePropValueNifiPropVar = niFiProperties.getProperty(NF_SENSITIVE_PROPS_KEY, DEFAULT_SENSITIVE_PROPS_KEY);
if (StringUtils.isBlank(sensitivePropAlgorithmVal)) {
throw new EncryptionException(NF_SENSITIVE_PROPS_ALGORITHM + "must bet set");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
index f73dce556210..5eceb522e87c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
@@ -45,10 +45,12 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
private final Path flowXmlPath;
private final StringEncryptor encryptor;
private final FlowConfigurationArchiveManager archiveManager;
+ private final NiFiProperties nifiProperties;
private static final Logger LOG = LoggerFactory.getLogger(StandardXMLFlowConfigurationDAO.class);
- public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor encryptor) throws IOException {
+ public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor encryptor, final NiFiProperties nifiProperties) throws IOException {
+ this.nifiProperties = nifiProperties;
final File flowXmlFile = flowXml.toFile();
if (!flowXmlFile.exists()) {
// createDirectories would throw an exception if the directory exists but is a symbolic link
@@ -64,7 +66,7 @@ public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor
this.flowXmlPath = flowXml;
this.encryptor = encryptor;
- this.archiveManager = new FlowConfigurationArchiveManager(flowXmlPath, NiFiProperties.getInstance());
+ this.archiveManager = new FlowConfigurationArchiveManager(flowXmlPath, nifiProperties);
}
@Override
@@ -77,7 +79,7 @@ public boolean isFlowPresent() {
public synchronized void load(final FlowController controller, final DataFlow dataFlow)
throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
- final FlowSynchronizer flowSynchronizer = new StandardFlowSynchronizer(encryptor);
+ final FlowSynchronizer flowSynchronizer = new StandardFlowSynchronizer(encryptor, nifiProperties);
controller.synchronize(flowSynchronizer, dataFlow);
if (StandardFlowSynchronizer.isEmpty(dataFlow)) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessorInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessorInitializationContext.java
index 885554563c88..3138fe9abf22 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessorInitializationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessorInitializationContext.java
@@ -16,10 +16,12 @@
*/
package org.apache.nifi.processor;
+import java.io.File;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.NodeTypeProvider;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.NiFiProperties;
public class StandardProcessorInitializationContext implements ProcessorInitializationContext {
@@ -27,12 +29,17 @@ public class StandardProcessorInitializationContext implements ProcessorInitiali
private final ComponentLog logger;
private final ControllerServiceProvider serviceProvider;
private final NodeTypeProvider nodeTypeProvider;
+ private final NiFiProperties nifiProperties;
- public StandardProcessorInitializationContext(final String identifier, final ComponentLog componentLog, final ControllerServiceProvider serviceProvider, NodeTypeProvider nodeTypeProvider) {
+ public StandardProcessorInitializationContext(
+ final String identifier, final ComponentLog componentLog,
+ final ControllerServiceProvider serviceProvider, final NodeTypeProvider nodeTypeProvider,
+ final NiFiProperties nifiProperties) {
this.identifier = identifier;
this.logger = componentLog;
this.serviceProvider = serviceProvider;
this.nodeTypeProvider = nodeTypeProvider;
+ this.nifiProperties = nifiProperties;
}
@Override
@@ -54,4 +61,19 @@ public ControllerServiceLookup getControllerServiceLookup() {
public NodeTypeProvider getNodeTypeProvider() {
return nodeTypeProvider;
}
+
+ @Override
+ public String getKerberosServicePrincipal() {
+ return nifiProperties.getKerberosServicePrincipal();
+ }
+
+ @Override
+ public File getKerberosServiceKeytab() {
+ return new File(nifiProperties.getKerberosKeytabLocation());
+ }
+
+ @Override
+ public File getKerberosConfigurationFile() {
+ return nifiProperties.getKerberosConfigurationFile();
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 43947ed4618c..1c9d182b89ac 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -76,8 +76,9 @@
import static java.util.Objects.requireNonNull;
/**
- * Represents the Root Process Group of a remote NiFi Instance. Holds information about that remote instance, as well as {@link IncomingPort}s and {@link OutgoingPort}s for communicating with the
- * remote instance.
+ * Represents the Root Process Group of a remote NiFi Instance. Holds
+ * information about that remote instance, as well as {@link IncomingPort}s and
+ * {@link OutgoingPort}s for communicating with the remote instance.
*/
public class StandardRemoteProcessGroup implements RemoteProcessGroup {
@@ -95,6 +96,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private final String protocol;
private final ProcessScheduler scheduler;
private final EventReporter eventReporter;
+ private final NiFiProperties nifiProperties;
private final AtomicReference name = new AtomicReference<>();
private final AtomicReference position = new AtomicReference<>();
@@ -115,7 +117,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private volatile String proxyUser;
private volatile String proxyPassword;
-
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
private final Lock readLock = rwLock.readLock();
private final Lock writeLock = rwLock.writeLock();
@@ -137,7 +138,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private final ScheduledExecutorService backgroundThreadExecutor;
public StandardRemoteProcessGroup(final String id, final String targetUri, final ProcessGroup processGroup,
- final FlowController flowController, final SSLContext sslContext) {
+ final FlowController flowController, final SSLContext sslContext, final NiFiProperties nifiProperties) {
+ this.nifiProperties = nifiProperties;
this.id = requireNonNull(id);
this.flowController = requireNonNull(flowController);
final URI uri;
@@ -174,7 +176,7 @@ public void reportEvent(final Severity severity, final String category, final St
final String sourceId = StandardRemoteProcessGroup.this.getIdentifier();
final String sourceName = StandardRemoteProcessGroup.this.getName();
bulletinRepository.addBulletin(BulletinFactory.createBulletin(groupId, sourceId, ComponentType.REMOTE_PROCESS_GROUP,
- sourceName, category, severity.name(), message));
+ sourceName, category, severity.name(), message));
}
};
@@ -400,8 +402,11 @@ public boolean containsInputPort(final String id) {
}
/**
- * Changes the currently configured input ports to the ports described in the given set. If any port is currently configured that is not in the set given, that port will be shutdown and removed.
- * If any port is currently not configured and is in the set given, that port will be instantiated and started.
+ * Changes the currently configured input ports to the ports described in
+ * the given set. If any port is currently configured that is not in the set
+ * given, that port will be shutdown and removed. If any port is currently
+ * not configured and is in the set given, that port will be instantiated
+ * and started.
*
* @param ports the new ports
*
@@ -450,10 +455,12 @@ public void setInputPorts(final Set ports) {
}
/**
- * Returns a boolean indicating whether or not an Output Port exists with the given ID
+ * Returns a boolean indicating whether or not an Output Port exists with
+ * the given ID
*
* @param id identifier of port
- * @return true if an Output Port exists with the given ID, false otherwise.
+ * @return true if an Output Port exists with the given ID,
+ * false otherwise.
*/
public boolean containsOutputPort(final String id) {
readLock.lock();
@@ -465,8 +472,11 @@ public boolean containsOutputPort(final String id) {
}
/**
- * Changes the currently configured output ports to the ports described in the given set. If any port is currently configured that is not in the set given, that port will be shutdown and removed.
- * If any port is currently not configured and is in the set given, that port will be instantiated and started.
+ * Changes the currently configured output ports to the ports described in
+ * the given set. If any port is currently configured that is not in the set
+ * given, that port will be shutdown and removed. If any port is currently
+ * not configured and is in the set given, that port will be instantiated
+ * and started.
*
* @param ports the new ports
*
@@ -519,7 +529,8 @@ public void setOutputPorts(final Set ports) {
*
*
* @throws NullPointerException if the given output Port is null
- * @throws IllegalStateException if the port does not belong to this remote process group
+ * @throws IllegalStateException if the port does not belong to this remote
+ * process group
*/
@Override
public void removeNonExistentPort(final RemoteGroupPort port) {
@@ -597,11 +608,13 @@ public void removeAllNonExistentPorts() {
}
/**
- * Adds an Output Port to this Remote Process Group that is described by this DTO.
+ * Adds an Output Port to this Remote Process Group that is described by
+ * this DTO.
*
* @param descriptor
*
- * @throws IllegalStateException if an Output Port already exists with the ID given by dto.getId()
+ * @throws IllegalStateException if an Output Port already exists with the
+ * ID given by dto.getId()
*/
private void addOutputPort(final RemoteProcessGroupPortDescriptor descriptor) {
writeLock.lock();
@@ -611,7 +624,7 @@ private void addOutputPort(final RemoteProcessGroupPortDescriptor descriptor) {
}
final StandardRemoteGroupPort port = new StandardRemoteGroupPort(descriptor.getId(), descriptor.getName(), getProcessGroup(),
- this, TransferDirection.RECEIVE, ConnectableType.REMOTE_OUTPUT_PORT, sslContext, scheduler);
+ this, TransferDirection.RECEIVE, ConnectableType.REMOTE_OUTPUT_PORT, sslContext, scheduler, nifiProperties);
outputPorts.put(descriptor.getId(), port);
if (descriptor.getConcurrentlySchedulableTaskCount() != null) {
@@ -627,7 +640,8 @@ private void addOutputPort(final RemoteProcessGroupPortDescriptor descriptor) {
/**
* @param portIdentifier the ID of the Port to send FlowFiles to
- * @return {@link RemoteGroupPort} that can be used to send FlowFiles to the port whose ID is given on the remote instance
+ * @return {@link RemoteGroupPort} that can be used to send FlowFiles to the
+ * port whose ID is given on the remote instance
*/
@Override
public RemoteGroupPort getInputPort(final String portIdentifier) {
@@ -644,7 +658,8 @@ public RemoteGroupPort getInputPort(final String portIdentifier) {
}
/**
- * @return a set of {@link OutgoingPort}s used for transmitting FlowFiles to the remote instance
+ * @return a set of {@link OutgoingPort}s used for transmitting FlowFiles to
+ * the remote instance
*/
@Override
public Set getInputPorts() {
@@ -659,11 +674,13 @@ public Set getInputPorts() {
}
/**
- * Adds an InputPort to this ProcessGroup that is described by the given DTO.
+ * Adds an InputPort to this ProcessGroup that is described by the given
+ * DTO.
*
* @param descriptor port descriptor
*
- * @throws IllegalStateException if an Input Port already exists with the ID given by the ID of the DTO.
+ * @throws IllegalStateException if an Input Port already exists with the ID
+ * given by the ID of the DTO.
*/
private void addInputPort(final RemoteProcessGroupPortDescriptor descriptor) {
writeLock.lock();
@@ -673,7 +690,7 @@ private void addInputPort(final RemoteProcessGroupPortDescriptor descriptor) {
}
final StandardRemoteGroupPort port = new StandardRemoteGroupPort(descriptor.getId(), descriptor.getName(), getProcessGroup(), this,
- TransferDirection.SEND, ConnectableType.REMOTE_INPUT_PORT, sslContext, scheduler);
+ TransferDirection.SEND, ConnectableType.REMOTE_INPUT_PORT, sslContext, scheduler, nifiProperties);
if (descriptor.getConcurrentlySchedulableTaskCount() != null) {
port.setMaxConcurrentTasks(descriptor.getConcurrentlySchedulableTaskCount());
@@ -703,7 +720,8 @@ public RemoteGroupPort getOutputPort(final String portIdentifier) {
}
/**
- * @return a set of {@link RemoteGroupPort}s used for receiving FlowFiles from the remote instance
+ * @return a set of {@link RemoteGroupPort}s used for receiving FlowFiles
+ * from the remote instance
*/
@Override
public Set getOutputPorts() {
@@ -772,10 +790,9 @@ private void refreshFlowContentsFromLocal() {
writeLock.lock();
try {
- final NiFiProperties props = NiFiProperties.getInstance();
- this.destinationSecure = props.isSiteToSiteSecure();
- this.listeningPort = props.getRemoteInputPort();
- this.listeningHttpPort = props.getRemoteInputHttpPort();
+ this.destinationSecure = nifiProperties.isSiteToSiteSecure();
+ this.listeningPort = nifiProperties.getRemoteInputPort();
+ this.listeningHttpPort = nifiProperties.getRemoteInputHttpPort();
refreshContentsTimestamp = System.currentTimeMillis();
} finally {
@@ -815,8 +832,7 @@ public void refreshFlowContents() throws CommunicationsException {
// perform the request
final ControllerDTO dto;
try (
- final SiteToSiteRestApiClient apiClient = getSiteToSiteRestApiClient();
- ){
+ final SiteToSiteRestApiClient apiClient = getSiteToSiteRestApiClient();) {
dto = apiClient.getController();
} catch (IOException e) {
writeLock.lock();
@@ -1138,7 +1154,7 @@ private class InitializationTask implements Runnable {
@Override
public void run() {
- try (final SiteToSiteRestApiClient apiClient = getSiteToSiteRestApiClient()){
+ try (final SiteToSiteRestApiClient apiClient = getSiteToSiteRestApiClient()) {
try {
final ControllerDTO dto = apiClient.getController();
@@ -1182,7 +1198,7 @@ public void run() {
logger.error("", e);
}
}
- */
+ */
authorizationIssue = e.getDescription();
} else if (e.getResponseCode() == FORBIDDEN_STATUS_CODE) {
@@ -1192,7 +1208,7 @@ public void run() {
logger.warn("{} When communicating with remote instance, got unexpected result. {}",
new Object[]{this, e.getMessage()});
authorizationIssue = "Unable to determine Site-to-Site availability.";
- }
+ }
}
} catch (final Exception e) {
@@ -1321,7 +1337,7 @@ public void verifyCanUpdate() {
}
private File getPeerPersistenceFile() {
- final File stateDir = NiFiProperties.getInstance().getPersistentStateDirectory();
+ final File stateDir = nifiProperties.getPersistentStateDirectory();
return new File(stateDir, getIdentifier() + ".peers");
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy
index 1ef622aab8f6..28b314c02443 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy
@@ -31,9 +31,10 @@ import spock.lang.Specification
import spock.lang.Unroll
class StandardFlowSynchronizerSpec extends Specification {
-
+
def setupSpec() {
- System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties"
+ def propFile = StandardFlowSynchronizerSpec.class.getResource("/nifi.properties").getFile()
+ System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile
}
def teardownSpec() {
@@ -64,7 +65,8 @@ class StandardFlowSynchronizerSpec extends Specification {
def Map connectionMocksById = [:]
def Map> bendPointPositionsByConnectionId = [:]
// the unit under test
- def flowSynchronizer = new StandardFlowSynchronizer(null)
+ def nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null)
+ def flowSynchronizer = new StandardFlowSynchronizer(null,nifiProperties)
when: "the flow is synchronized with the current state of the controller"
flowSynchronizer.sync controller, proposedFlow, null
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
index 986fba4f1067..79b59d76b4cc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
@@ -74,7 +74,7 @@ public static void setupSuite() {
@Before
public void setup() throws Exception {
- properties = NiFiProperties.getInstance();
+ properties = NiFiProperties.createBasicNiFiProperties(null, null);
variableRegistry = new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths());
mockFlowFileEventRepository = mock(FlowFileEventRepository.class);
authorizer = mock(Authorizer.class);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
index 15a8267ec94d..6c52def84a87 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
@@ -49,10 +49,9 @@ public class TestFileSystemSwapManager {
@Test
public void testBackwardCompatible() throws IOException {
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
try (final InputStream fis = new FileInputStream(new File("src/test/resources/old-swap-file.swap"));
- final DataInputStream in = new DataInputStream(new BufferedInputStream(fis))) {
+ final DataInputStream in = new DataInputStream(new BufferedInputStream(fis))) {
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
@@ -91,7 +90,7 @@ public void testRoundTripSerializeDeserialize() throws IOException {
final SwapContents swappedIn;
try (final FileInputStream fis = new FileInputStream(swapFile);
- final DataInputStream dis = new DataInputStream(fis)) {
+ final DataInputStream dis = new DataInputStream(fis)) {
swappedIn = FileSystemSwapManager.deserializeFlowFiles(dis, swapLocation, flowFileQueue, Mockito.mock(ResourceClaimManager.class));
}
@@ -113,7 +112,6 @@ public void testRoundTripSerializeDeserialize() throws IOException {
}
}
-
public class NopResourceClaimManager implements ResourceClaimManager {
@Override
@@ -162,8 +160,8 @@ public void freeze(ResourceClaim claim) {
}
}
-
private static class TestFlowFile implements FlowFileRecord {
+
private static final AtomicLong idGenerator = new AtomicLong(0L);
private final long id = idGenerator.getAndIncrement();
@@ -172,13 +170,11 @@ private static class TestFlowFile implements FlowFileRecord {
private final Map attributes;
private final long size;
-
public TestFlowFile(final Map attributes, final long size) {
this.attributes = attributes;
this.size = size;
}
-
@Override
public long getId() {
return id;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
index 71af93f10b67..602ddce7d6dc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.controller;
import org.apache.commons.io.IOUtils;
@@ -46,7 +45,9 @@
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
import java.util.LinkedHashSet;
+import java.util.Map;
import java.util.Set;
import org.apache.nifi.util.FileBasedVariableRegistry;
@@ -65,21 +66,22 @@ public class TestFlowController {
private FlowFileEventRepository flowFileEventRepo;
private AuditService auditService;
private StringEncryptor encryptor;
- private NiFiProperties properties;
+ private NiFiProperties nifiProperties;
private BulletinRepository bulletinRepo;
private VariableRegistry variableRegistry;
@Before
public void setup() {
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFlowController.class.getResource("/nifi.properties").getFile());
flowFileEventRepo = Mockito.mock(FlowFileEventRepository.class);
auditService = Mockito.mock(AuditService.class);
- encryptor = StringEncryptor.createEncryptor();
- properties = NiFiProperties.getInstance();
- properties.setProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, MockProvenanceRepository.class.getName());
- properties.setProperty("nifi.remote.input.socket.port", "");
- properties.setProperty("nifi.remote.input.secure", "");
+ final Map otherProps = new HashMap<>();
+ otherProps.put(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, MockProvenanceRepository.class.getName());
+ otherProps.put("nifi.remote.input.socket.port", "");
+ otherProps.put("nifi.remote.input.secure", "");
+ nifiProperties = NiFiProperties.createBasicNiFiProperties(null, otherProps);
+ encryptor = StringEncryptor.createEncryptor(nifiProperties);
User user1 = new User.Builder().identifier("user-id-1").identity("user-1").build();
User user2 = new User.Builder().identifier("user-id-2").identity("user-2").build();
@@ -118,12 +120,12 @@ public void setup() {
policies1.add(policy2);
authorizer = new MockPolicyBasedAuthorizer(groups1, users1, policies1);
- variableRegistry = new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths());
+ variableRegistry = new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths());
bulletinRepo = Mockito.mock(BulletinRepository.class);
- controller = FlowController.createStandaloneInstance(flowFileEventRepo, properties, authorizer, auditService, encryptor, bulletinRepo,variableRegistry);
+ controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer, auditService, encryptor, bulletinRepo, variableRegistry);
- standardFlowSynchronizer = new StandardFlowSynchronizer(StringEncryptor.createEncryptor());
+ standardFlowSynchronizer = new StandardFlowSynchronizer(StringEncryptor.createEncryptor(nifiProperties), nifiProperties);
}
@After
@@ -271,7 +273,7 @@ public void testSynchronizeFlowWhenCurrentAuthorizationsAreEmptyAndProposedAreNo
assertNotEquals(authFingerprint, authorizer.getFingerprint());
controller.shutdown(true);
- controller = FlowController.createStandaloneInstance(flowFileEventRepo, properties, authorizer, auditService, encryptor, bulletinRepo,variableRegistry);
+ controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer, auditService, encryptor, bulletinRepo, variableRegistry);
controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
assertEquals(authFingerprint, authorizer.getFingerprint());
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
index 3379f793f4b8..f6dc88eacb77 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
@@ -41,6 +41,7 @@
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.NiFiProperties;
import org.junit.Assert;
import org.junit.Test;
@@ -48,11 +49,11 @@ public class TestStandardProcessorNode {
@Test(timeout = 10000)
public void testStart() throws InterruptedException {
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestStandardProcessorNode.class.getResource("/conf/nifi.properties").getFile());
final ProcessorThatThrowsExceptionOnScheduled processor = new ProcessorThatThrowsExceptionOnScheduled();
final String uuid = UUID.randomUUID().toString();
- final StandardProcessorNode procNode = new StandardProcessorNode(processor, uuid, createValidationContextFactory(), null, null);
+ final StandardProcessorNode procNode = new StandardProcessorNode(processor, uuid, createValidationContextFactory(), null, null, NiFiProperties.createBasicNiFiProperties(null, null));
final ScheduledExecutorService taskScheduler = new FlowEngine(2, "TestStandardProcessorNode", true);
final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, null);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
index 2a82aedaf89a..af33ccb2a7d6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
@@ -55,6 +55,8 @@
import ch.qos.logback.classic.Logger;
import ch.qos.logback.classic.spi.ILoggingEvent;
import ch.qos.logback.core.read.ListAppender;
+import java.util.HashMap;
+import java.util.Map;
public class TestFileSystemRepository {
@@ -65,14 +67,16 @@ public class TestFileSystemRepository {
private FileSystemRepository repository = null;
private StandardResourceClaimManager claimManager = null;
private final File rootFile = new File("target/content_repository");
+ private NiFiProperties nifiProperties;
@Before
public void setup() throws IOException {
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile());
+ nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
if (rootFile.exists()) {
DiskUtils.deleteRecursively(rootFile);
}
- repository = new FileSystemRepository();
+ repository = new FileSystemRepository(nifiProperties);
claimManager = new StandardResourceClaimManager();
repository.initialize(claimManager);
repository.purge();
@@ -94,49 +98,39 @@ public void testMinimalArchiveCleanupIntervalHonoredAndLogged() throws Exception
testAppender.setName("Test");
testAppender.start();
root.addAppender(testAppender);
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY, "1 millis");
+ final NiFiProperties localProps = NiFiProperties.createBasicNiFiProperties(null, addProps);
+ repository = new FileSystemRepository(localProps);
+ repository.initialize(new StandardResourceClaimManager());
+ repository.purge();
- final NiFiProperties properties = NiFiProperties.getInstance();
- final String originalCleanupFreq = properties.getProperty(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY);
- properties.setProperty(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY, "1 millis");
- try {
- repository = new FileSystemRepository();
- repository.initialize(new StandardResourceClaimManager());
- repository.purge();
-
-
- boolean messageFound = false;
- String message = "The value of nifi.content.repository.archive.cleanup.frequency property "
+ boolean messageFound = false;
+ String message = "The value of nifi.content.repository.archive.cleanup.frequency property "
+ "is set to '1 millis' which is below the allowed minimum of 1 second (1000 milliseconds). "
+ "Minimum value of 1 sec will be used as scheduling interval for archive cleanup task.";
- for (ILoggingEvent event : testAppender.list) {
- String actualMessage = event.getFormattedMessage();
- if (actualMessage.equals(message)) {
- assertEquals(event.getLevel(), Level.WARN);
- messageFound = true;
- break;
- }
- }
- assertTrue(messageFound);
- } finally {
- if (originalCleanupFreq == null) {
- properties.remove(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY);
- } else {
- properties.setProperty(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY, originalCleanupFreq);
+ for (ILoggingEvent event : testAppender.list) {
+ String actualMessage = event.getFormattedMessage();
+ if (actualMessage.equals(message)) {
+ assertEquals(event.getLevel(), Level.WARN);
+ messageFound = true;
+ break;
}
}
+ assertTrue(messageFound);
}
@Test
public void testBogusFile() throws IOException {
repository.shutdown();
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile());
File bogus = new File(rootFile, "bogus");
try {
bogus.mkdir();
bogus.setReadable(false);
- repository = new FileSystemRepository();
+ repository = new FileSystemRepository(nifiProperties);
repository.initialize(new StandardResourceClaimManager());
} finally {
bogus.setReadable(true);
@@ -198,7 +192,7 @@ public void testResourceClaimNotReusedAfterRestart() throws IOException, Interru
repository.shutdown();
Thread.sleep(1000L);
- repository = new FileSystemRepository();
+ repository = new FileSystemRepository(nifiProperties);
repository.initialize(new StandardResourceClaimManager());
repository.purge();
@@ -206,7 +200,6 @@ public void testResourceClaimNotReusedAfterRestart() throws IOException, Interru
assertNotSame(claim1.getResourceClaim(), claim2.getResourceClaim());
}
-
@Test
public void testWriteWithNoContent() throws IOException {
final ContentClaim claim1 = repository.create(false);
@@ -314,7 +307,6 @@ public void testImportFromFile() throws IOException {
assertTrue(Arrays.equals(expected, baos.toByteArray()));
}
-
@Test
public void testImportFromStream() throws IOException {
final ContentClaim claim = repository.create(false);
@@ -452,7 +444,7 @@ public void testMarkDestructableDoesNotArchiveIfStreamOpenAndWrittenTo() throws
// We are creating our own 'local' repository in this test so shut down the one created in the setup() method
shutdown();
- repository = new FileSystemRepository() {
+ repository = new FileSystemRepository(nifiProperties) {
@Override
protected boolean archive(Path curPath) throws IOException {
archivedPaths.add(curPath);
@@ -494,7 +486,6 @@ protected boolean archive(Path curPath) throws IOException {
}
}
-
@Test
public void testWriteCannotProvideNullOutput() throws IOException {
FileSystemRepository repository = null;
@@ -504,7 +495,7 @@ public void testWriteCannotProvideNullOutput() throws IOException {
// We are creating our own 'local' repository in this test so shut down the one created in the setup() method
shutdown();
- repository = new FileSystemRepository() {
+ repository = new FileSystemRepository(nifiProperties) {
@Override
protected boolean archive(Path curPath) throws IOException {
if (getOpenStreamCount() > 0) {
@@ -545,23 +536,33 @@ protected boolean archive(Path curPath) throws IOException {
}
/**
- * We have encountered a situation where the File System Repo is moving files to archive and then eventually
- * aging them off while there is still an open file handle. This test is meant to replicate the conditions under
+ * We have encountered a situation where the File System Repo is moving
+ * files to archive and then eventually aging them off while there is still
+ * an open file handle. This test is meant to replicate the conditions under
* which this would happen and verify that it is fixed.
*
- * The condition that caused this appears to be that a Process Session created a Content Claim and then did not write
- * to it. It then decremented the claimant count (which reduced the count to 0). This was likely due to creating the
- * claim in ProcessSession.write(FlowFile, StreamCallback) and then having an Exception thrown when the Process Session
- * attempts to read the current Content Claim. In this case, it would not ever get to the point of calling
- * FileSystemRepository.write().
+ * The condition that caused this appears to be that a Process Session
+ * created a Content Claim and then did not write to it. It then decremented
+ * the claimant count (which reduced the count to 0). This was likely due to
+ * creating the claim in ProcessSession.write(FlowFile, StreamCallback) and
+ * then having an Exception thrown when the Process Session attempts to read
+ * the current Content Claim. In this case, it would not ever get to the
+ * point of calling FileSystemRepository.write().
*
- * The above sequence of events is problematic because calling FileSystemRepository.create() will remove the Resource Claim
- * from the 'writable claims queue' and expects that we will write to it. When we call FileSystemRepository.write() with that
- * Resource Claim, we return an OutputStream that, when closed, will take care of adding the Resource Claim back to the
- * 'writable claims queue' or otherwise close the FileOutputStream that is open for that Resource Claim. If FileSystemRepository.write()
- * is never called, or if the OutputStream returned by that method is never closed, but the Content Claim is then decremented to 0,
- * we can get into a situation where we do archive the content (because the claimant count is 0 and it is not in the 'writable claims queue')
- * and then eventually age it off, without ever closing the OutputStream. We need to ensure that we do always close that Output Stream.
+ * The above sequence of events is problematic because calling
+ * FileSystemRepository.create() will remove the Resource Claim from the
+ * 'writable claims queue' and expects that we will write to it. When we
+ * call FileSystemRepository.write() with that Resource Claim, we return an
+ * OutputStream that, when closed, will take care of adding the Resource
+ * Claim back to the 'writable claims queue' or otherwise close the
+ * FileOutputStream that is open for that Resource Claim. If
+ * FileSystemRepository.write() is never called, or if the OutputStream
+ * returned by that method is never closed, but the Content Claim is then
+ * decremented to 0, we can get into a situation where we do archive the
+ * content (because the claimant count is 0 and it is not in the 'writable
+ * claims queue') and then eventually age it off, without ever closing the
+ * OutputStream. We need to ensure that we do always close that Output
+ * Stream.
*/
@Test
public void testMarkDestructableDoesNotArchiveIfStreamOpenAndNotWrittenTo() throws IOException, InterruptedException {
@@ -572,7 +573,7 @@ public void testMarkDestructableDoesNotArchiveIfStreamOpenAndNotWrittenTo() thro
// We are creating our own 'local' repository in this test so shut down the one created in the setup() method
shutdown();
- repository = new FileSystemRepository() {
+ repository = new FileSystemRepository(nifiProperties) {
@Override
protected boolean archive(Path curPath) throws IOException {
if (getOpenStreamCount() > 0) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
index 658e8c0f19f4..8d398aaefbac 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
@@ -80,6 +80,7 @@
import org.apache.nifi.provenance.ProvenanceEventRepository;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.NiFiProperties;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -138,7 +139,7 @@ private void rmDir(final File dir) {
public void setup() throws IOException {
resourceClaimManager = new StandardResourceClaimManager();
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestStandardProcessSession.class.getResource("/conf/nifi.properties").getFile());
final FlowFileEventRepository flowFileEventRepo = Mockito.mock(FlowFileEventRepository.class);
final CounterRepository counterRepo = Mockito.mock(CounterRepository.class);
provenanceRepo = new MockProvenanceRepository();
@@ -192,9 +193,9 @@ public Set answer(final InvocationOnMock invocation) throws Throwabl
final Relationship relationship = (Relationship) arguments[0];
if (relationship == Relationship.SELF) {
return Collections.emptySet();
- } else if (relationship == FAKE_RELATIONSHIP || relationship.equals(FAKE_RELATIONSHIP) ){
+ } else if (relationship == FAKE_RELATIONSHIP || relationship.equals(FAKE_RELATIONSHIP)) {
return null;
- }else {
+ } else {
return new HashSet<>(connList);
}
}
@@ -213,10 +214,10 @@ public Set answer(final InvocationOnMock invocation) throws Throwabl
@Test
public void testAppendToChildThrowsIOExceptionThenRemove() throws IOException {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .id(1000L)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .id(1000L)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile original = session.get();
assertNotNull(original);
@@ -245,10 +246,10 @@ public void testAppendToChildThrowsIOExceptionThenRemove() throws IOException {
@Test
public void testWriteForChildThrowsIOExceptionThenRemove() throws IOException {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .id(1000L)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .id(1000L)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile original = session.get();
assertNotNull(original);
@@ -274,7 +275,6 @@ public void testWriteForChildThrowsIOExceptionThenRemove() throws IOException {
assertEquals(0, numClaims);
}
-
@Test
public void testModifyContentThenRollback() throws IOException {
assertEquals(0, contentRepo.getExistingClaims().size());
@@ -283,10 +283,10 @@ public void testModifyContentThenRollback() throws IOException {
assertEquals(1, contentRepo.getExistingClaims().size());
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .contentClaim(claim)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .contentClaim(claim)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile flowFile = session.get();
@@ -384,10 +384,10 @@ private void assertDisabled(final InputStream inputStream) {
public void testAppendAfterSessionClosesStream() throws IOException {
final ContentClaim claim = contentRepo.create(false);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .contentClaim(claim)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .contentClaim(claim)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile flowFile = session.get();
assertNotNull(flowFile);
@@ -405,10 +405,10 @@ public void process(final OutputStream outputStream) throws IOException {
public void testExportTo() throws IOException {
final ContentClaim claim = contentRepo.create(false);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .contentClaim(claim)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .contentClaim(claim)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile flowFile = session.get();
assertNotNull(flowFile);
@@ -440,15 +440,15 @@ public void process(OutputStream out) throws IOException {
public void testReadAfterSessionClosesStream() throws IOException {
final ContentClaim claim = contentRepo.create(false);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .contentClaim(claim)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .contentClaim(claim)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
final FlowFile flowFile = session.get();
assertNotNull(flowFile);
final AtomicReference inputStreamHolder = new AtomicReference<>(null);
- session.read(flowFile, true , new InputStreamCallback() {
+ session.read(flowFile, true, new InputStreamCallback() {
@Override
public void process(final InputStream inputStream) throws IOException {
inputStreamHolder.set(inputStream);
@@ -461,10 +461,10 @@ public void process(final InputStream inputStream) throws IOException {
public void testStreamAfterSessionClosesStream() throws IOException {
final ContentClaim claim = contentRepo.create(false);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .contentClaim(claim)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .contentClaim(claim)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile flowFile = session.get();
assertNotNull(flowFile);
@@ -485,10 +485,10 @@ public void process(final InputStream input, final OutputStream output) throws I
public void testWriteAfterSessionClosesStream() throws IOException {
final ContentClaim claim = contentRepo.create(false);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .contentClaim(claim)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .contentClaim(claim)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile flowFile = session.get();
assertNotNull(flowFile);
@@ -506,9 +506,9 @@ public void process(final OutputStream out) throws IOException {
public void testCreateThenRollbackRemovesContent() throws IOException {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
final StreamCallback nop = new StreamCallback() {
@@ -536,9 +536,9 @@ public void process(InputStream in, OutputStream out) throws IOException {
@Test
public void testForksNotEmittedIfFilesDeleted() throws IOException {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
@@ -553,9 +553,9 @@ public void testForksNotEmittedIfFilesDeleted() throws IOException {
@Test
public void testProvenanceEventsEmittedForForkIfNotRemoved() throws IOException {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
@@ -570,9 +570,9 @@ public void testProvenanceEventsEmittedForForkIfNotRemoved() throws IOException
@Test
public void testProvenanceEventsEmittedForRemove() throws IOException {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
@@ -590,10 +590,10 @@ public void testProvenanceEventsEmittedForRemove() throws IOException {
public void testUuidAttributeCannotBeUpdated() {
String originalUuid = "11111111-1111-1111-1111-111111111111";
final FlowFileRecord flowFileRecord1 = new StandardFlowFileRecord.Builder()
- .id(1L)
- .addAttribute("uuid", originalUuid)
- .entryDate(System.currentTimeMillis())
- .build();
+ .id(1L)
+ .addAttribute("uuid", originalUuid)
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord1);
@@ -625,16 +625,16 @@ public void testUuidAttributeCannotBeUpdated() {
@Test
public void testUpdateAttributesThenJoin() throws IOException {
final FlowFileRecord flowFileRecord1 = new StandardFlowFileRecord.Builder()
- .id(1L)
- .addAttribute("uuid", "11111111-1111-1111-1111-111111111111")
- .entryDate(System.currentTimeMillis())
- .build();
+ .id(1L)
+ .addAttribute("uuid", "11111111-1111-1111-1111-111111111111")
+ .entryDate(System.currentTimeMillis())
+ .build();
final FlowFileRecord flowFileRecord2 = new StandardFlowFileRecord.Builder()
- .id(2L)
- .addAttribute("uuid", "22222222-2222-2222-2222-222222222222")
- .entryDate(System.currentTimeMillis())
- .build();
+ .id(2L)
+ .addAttribute("uuid", "22222222-2222-2222-2222-222222222222")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord1);
flowFileQueue.put(flowFileRecord2);
@@ -698,9 +698,9 @@ public void testUpdateAttributesThenJoin() throws IOException {
@Test
public void testForkOneToOneReported() throws IOException {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord);
@@ -815,11 +815,11 @@ public void process(final InputStream in, OutputStream out) throws IOException {
@Test
public void testMissingFlowFileExceptionThrownWhenUnableToReadData() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
- .size(1L)
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .size(1L)
+ .build();
flowFileQueue.put(flowFileRecord);
// attempt to read the data.
@@ -894,17 +894,16 @@ public void process(OutputStream out) throws IOException {
assertEquals(1, countAfterAppend);
}
-
@Test
@SuppressWarnings("unchecked")
public void testExpireDecrementsClaimsOnce() throws IOException {
final ContentClaim contentClaim = contentRepo.create(false);
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(contentClaim)
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(contentClaim)
+ .build();
Mockito.doAnswer(new Answer>() {
int iterations = 0;
@@ -930,7 +929,7 @@ public List answer(InvocationOnMock invocation) throws Throwable
public void testManyFilesOpened() throws IOException {
StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000];
- for(int i = 0; i<70000;i++){
+ for (int i = 0; i < 70000; i++) {
standardProcessSessions[i] = new StandardProcessSession(context);
FlowFile flowFile = standardProcessSessions[i].create();
@@ -950,12 +949,12 @@ public void process(final InputStream in) throws IOException {
StreamUtils.fillBuffer(in, buff);
}
});
- } catch (Exception e){
- System.out.println("Failed at file:"+i);
+ } catch (Exception e) {
+ System.out.println("Failed at file:" + i);
throw e;
}
- if(i%1000==0){
- System.out.println("i:"+i);
+ if (i % 1000 == 0) {
+ System.out.println("i:" + i);
}
}
}
@@ -963,11 +962,11 @@ public void process(final InputStream in) throws IOException {
@Test
public void testMissingFlowFileExceptionThrownWhenUnableToReadDataStreamCallback() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
- .size(1L)
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .size(1L)
+ .build();
flowFileQueue.put(flowFileRecord);
// attempt to read the data.
@@ -987,10 +986,10 @@ public void process(InputStream in, OutputStream out) throws IOException {
@Test
public void testContentNotFoundExceptionThrownWhenUnableToReadDataStreamCallbackOffsetTooLarge() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .build();
flowFileQueue.put(flowFileRecord);
FlowFile ff1 = session.get();
@@ -1003,12 +1002,12 @@ public void process(OutputStream out) throws IOException {
session.commit();
final FlowFileRecord flowFileRecord2 = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
- .contentClaimOffset(1000L)
- .size(1000L)
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .contentClaimOffset(1000L)
+ .size(1000L)
+ .build();
flowFileQueue.put(flowFileRecord2);
// attempt to read the data.
@@ -1028,10 +1027,10 @@ public void process(InputStream in, OutputStream out) throws IOException {
@Test
public void testContentNotFoundExceptionThrownWhenUnableToReadDataOffsetTooLarge() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
- .build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .build();
flowFileQueue.put(flowFileRecord);
@@ -1045,11 +1044,10 @@ public void process(OutputStream out) throws IOException {
session.commit();
final FlowFileRecord flowFileRecord2 = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
-
- .contentClaimOffset(1000L).size(1L).build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .contentClaimOffset(1000L).size(1L).build();
flowFileQueue.put(flowFileRecord2);
// attempt to read the data.
@@ -1110,15 +1108,13 @@ public void process(final InputStream in) throws IOException {
}
}
-
@Test
public void testCommitFailureRequeuesFlowFiles() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
-
- .contentClaimOffset(0L).size(0L).build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .contentClaimOffset(0L).size(0L).build();
flowFileQueue.put(flowFileRecord);
final FlowFile originalFlowFile = session.get();
@@ -1152,11 +1148,10 @@ public void process(OutputStream out) throws IOException {
@Test
public void testRollbackAfterCheckpoint() {
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
-
- .contentClaimOffset(0L).size(0L).build();
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .contentClaim(new StandardContentClaim(resourceClaimManager.newResourceClaim("x", "x", "0", true), 0L))
+ .contentClaimOffset(0L).size(0L).build();
flowFileQueue.put(flowFileRecord);
final FlowFile originalFlowFile = session.get();
@@ -1246,9 +1241,9 @@ public void process(OutputStream out) throws IOException {
@Test
public void testContentModifiedEmittedAndNotAttributesModified() throws IOException {
final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder()
- .id(1L)
- .addAttribute("uuid", "000000000000-0000-0000-0000-00000000")
- .build();
+ .id(1L)
+ .addAttribute("uuid", "000000000000-0000-0000-0000-00000000")
+ .build();
this.flowFileQueue.put(flowFile);
FlowFile existingFlowFile = session.get();
@@ -1273,9 +1268,9 @@ public void process(OutputStream out) throws IOException {
public void testGetWithCount() {
for (int i = 0; i < 8; i++) {
final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder()
- .id(i)
- .addAttribute("uuid", "000000000000-0000-0000-0000-0000000" + i)
- .build();
+ .id(i)
+ .addAttribute("uuid", "000000000000-0000-0000-0000-0000000" + i)
+ .build();
this.flowFileQueue.put(flowFile);
}
@@ -1286,9 +1281,9 @@ public void testGetWithCount() {
@Test
public void testAttributesModifiedEmitted() throws IOException {
final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder()
- .id(1L)
- .addAttribute("uuid", "000000000000-0000-0000-0000-00000000")
- .build();
+ .id(1L)
+ .addAttribute("uuid", "000000000000-0000-0000-0000-00000000")
+ .build();
this.flowFileQueue.put(flowFile);
FlowFile existingFlowFile = session.get();
@@ -1351,11 +1346,11 @@ public void testOpenMultipleInputStreamsToFlowFile() throws IOException {
}
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
- .contentClaim(claim)
- .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
- .entryDate(System.currentTimeMillis())
- .size(12L)
- .build();
+ .contentClaim(claim)
+ .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")
+ .entryDate(System.currentTimeMillis())
+ .size(12L)
+ .build();
flowFileQueue.put(flowFileRecord);
final FlowFile flowFile = session.get();
@@ -1377,10 +1372,10 @@ public void testOpenMultipleInputStreamsToFlowFile() throws IOException {
@Test
public void testTransferUnknownRelationship() {
final FlowFileRecord flowFileRecord1 = new StandardFlowFileRecord.Builder()
- .id(1L)
- .addAttribute("uuid", "11111111-1111-1111-1111-111111111111")
- .entryDate(System.currentTimeMillis())
- .build();
+ .id(1L)
+ .addAttribute("uuid", "11111111-1111-1111-1111-111111111111")
+ .entryDate(System.currentTimeMillis())
+ .build();
flowFileQueue.put(flowFileRecord1);
@@ -1403,6 +1398,7 @@ public void testTransferUnknownRelationship() {
}
private static class MockFlowFileRepository implements FlowFileRepository {
+
private boolean failOnUpdate = false;
private final AtomicLong idGenerator = new AtomicLong(0L);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestVolatileContentRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestVolatileContentRepository.java
index 5733164b125c..feed31a9ed28 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestVolatileContentRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestVolatileContentRepository.java
@@ -16,8 +16,6 @@
*/
package org.apache.nifi.controller.repository;
-import org.apache.nifi.controller.repository.VolatileContentRepository;
-
import static org.junit.Assert.assertEquals;
import java.io.ByteArrayOutputStream;
@@ -25,6 +23,8 @@
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
@@ -52,11 +52,11 @@ public void setup() {
@Test
public void testRedirects() throws IOException {
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties");
- final NiFiProperties props = NiFiProperties.getInstance();
- props.setProperty(VolatileContentRepository.MAX_SIZE_PROPERTY, "10 MB");
-
- final VolatileContentRepository contentRepo = new VolatileContentRepository();
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestVolatileContentRepository.class.getResource("/conf/nifi.properties").getFile());
+ final Map addProps = new HashMap<>();
+ addProps.put(VolatileContentRepository.MAX_SIZE_PROPERTY, "10 MB");
+ final NiFiProperties nifiProps = NiFiProperties.createBasicNiFiProperties(null, addProps);
+ final VolatileContentRepository contentRepo = new VolatileContentRepository(nifiProps);
contentRepo.initialize(claimManager);
final ContentClaim claim = contentRepo.create(true);
final OutputStream out = contentRepo.write(claim);
@@ -106,11 +106,12 @@ public void testRedirects() throws IOException {
@Test
public void testMemoryIsFreed() throws IOException, InterruptedException {
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties");
- final NiFiProperties props = NiFiProperties.getInstance();
- props.setProperty(VolatileContentRepository.MAX_SIZE_PROPERTY, "11 MB");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestVolatileContentRepository.class.getResource("/conf/nifi.properties").getFile());
+ final Map addProps = new HashMap<>();
+ addProps.put(VolatileContentRepository.MAX_SIZE_PROPERTY, "11 MB");
+ final NiFiProperties nifiProps = NiFiProperties.createBasicNiFiProperties(null, addProps);
+ final VolatileContentRepository contentRepo = new VolatileContentRepository(nifiProps);
- final VolatileContentRepository contentRepo = new VolatileContentRepository();
contentRepo.initialize(claimManager);
final byte[] oneK = new byte[1024];
@@ -151,11 +152,11 @@ public void run() {
@Test
public void testSimpleReadWrite() throws IOException {
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties");
- final NiFiProperties props = NiFiProperties.getInstance();
- props.setProperty(VolatileContentRepository.MAX_SIZE_PROPERTY, "10 MB");
-
- final VolatileContentRepository contentRepo = new VolatileContentRepository();
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestVolatileContentRepository.class.getResource("/conf/nifi.properties").getFile());
+ final Map addProps = new HashMap<>();
+ addProps.put(VolatileContentRepository.MAX_SIZE_PROPERTY, "11 MB");
+ final NiFiProperties nifiProps = NiFiProperties.createBasicNiFiProperties(null, addProps);
+ final VolatileContentRepository contentRepo = new VolatileContentRepository(nifiProps);
contentRepo.initialize(claimManager);
final ContentClaim claim = contentRepo.create(true);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
index 51b654f7b08a..674f78fb8ff6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
@@ -47,6 +47,7 @@
import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
import org.apache.nifi.controller.swap.StandardSwapContents;
import org.apache.nifi.controller.swap.StandardSwapSummary;
+import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.file.FileUtils;
import org.junit.Before;
import org.junit.BeforeClass;
@@ -59,7 +60,7 @@ public class TestWriteAheadFlowFileRepository {
@BeforeClass
public static void setupProperties() {
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestWriteAheadFlowFileRepository.class.getResource("/conf/nifi.properties").getFile());
}
@Before
@@ -95,26 +96,26 @@ public void testResourceClaimsIncremented() throws IOException {
// Create a flowfile repo, update it once with a FlowFile that points to one resource claim. Then,
// indicate that a FlowFile was swapped out. We should then be able to recover these FlowFiles and the
// resource claims' counts should be updated for both the swapped out FlowFile and the non-swapped out FlowFile
- try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository()) {
+ try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null))) {
repo.initialize(claimManager);
repo.loadFlowFiles(queueProvider, -1L);
// Create a Repository Record that indicates that a FlowFile was created
final FlowFileRecord flowFile1 = new StandardFlowFileRecord.Builder()
- .id(1L)
- .addAttribute("uuid", "11111111-1111-1111-1111-111111111111")
- .contentClaim(claim1)
- .build();
+ .id(1L)
+ .addAttribute("uuid", "11111111-1111-1111-1111-111111111111")
+ .contentClaim(claim1)
+ .build();
final StandardRepositoryRecord rec1 = new StandardRepositoryRecord(queue);
rec1.setWorking(flowFile1);
rec1.setDestination(queue);
// Create a Record that we can swap out
final FlowFileRecord flowFile2 = new StandardFlowFileRecord.Builder()
- .id(2L)
- .addAttribute("uuid", "11111111-1111-1111-1111-111111111112")
- .contentClaim(claim2)
- .build();
+ .id(2L)
+ .addAttribute("uuid", "11111111-1111-1111-1111-111111111112")
+ .contentClaim(claim2)
+ .build();
final StandardRepositoryRecord rec2 = new StandardRepositoryRecord(queue);
rec2.setWorking(flowFile2);
@@ -129,9 +130,8 @@ public void testResourceClaimsIncremented() throws IOException {
repo.swapFlowFilesOut(Collections.singletonList(flowFile2), queue, swapLocation);
}
-
final ResourceClaimManager recoveryClaimManager = new StandardResourceClaimManager();
- try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository()) {
+ try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null))) {
repo.initialize(recoveryClaimManager);
final long largestId = repo.loadFlowFiles(queueProvider, 0L);
@@ -162,7 +162,7 @@ public void testRestartWithOneRecord() throws IOException {
FileUtils.deleteFile(path.toFile(), true);
}
- final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository();
+ final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null));
repo.initialize(new StandardResourceClaimManager());
final TestQueueProvider queueProvider = new TestQueueProvider();
@@ -216,7 +216,7 @@ public Object answer(final InvocationOnMock invocation) throws Throwable {
repo.close();
// restore
- final WriteAheadFlowFileRepository repo2 = new WriteAheadFlowFileRepository();
+ final WriteAheadFlowFileRepository repo2 = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null));
repo2.initialize(new StandardResourceClaimManager());
repo2.loadFlowFiles(queueProvider, 0L);
@@ -231,6 +231,7 @@ public Object answer(final InvocationOnMock invocation) throws Throwable {
}
private static class TestQueueProvider implements QueueProvider {
+
private List connectionList = new ArrayList<>();
public void addConnection(final Connection connection) {
@@ -249,6 +250,7 @@ public Collection getAllQueues() {
}
private static class MockFlowFileSwapManager implements FlowFileSwapManager {
+
private final Map>> swappedRecords = new HashMap<>();
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
index 602efe72a1fe..2ebe8cf9faff 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
@@ -56,8 +56,10 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
+import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import java.util.Random;
import java.util.UUID;
import java.util.concurrent.CountDownLatch;
@@ -83,11 +85,7 @@ public class TestProcessorLifecycle {
@Before
public void before() throws Exception {
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
- NiFiProperties.getInstance().setProperty(NiFiProperties.ADMINISTRATIVE_YIELD_DURATION, "1 sec");
- NiFiProperties.getInstance().setProperty(NiFiProperties.STATE_MANAGEMENT_CONFIG_FILE, "target/test-classes/state-management.xml");
- NiFiProperties.getInstance().setProperty(NiFiProperties.STATE_MANAGEMENT_LOCAL_PROVIDER_ID, "local-provider");
- fc = this.buildFlowControllerForTest();
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestProcessorLifecycle.class.getResource("/nifi.properties").getFile());
}
@After
@@ -99,6 +97,7 @@ public void after() throws Exception {
@Test
public void validateEnableOperation() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
final ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(),
@@ -117,9 +116,9 @@ public void validateEnableOperation() throws Exception {
assertEquals(ScheduledState.DISABLED, testProcNode.getPhysicalScheduledState());
}
-
@Test
public void validateDisableOperation() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
final ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(),
@@ -145,6 +144,7 @@ public void validateDisableOperation() throws Exception {
*/
@Test
public void validateIdempotencyOfProcessorStartOperation() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
final ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -170,6 +170,7 @@ public void validateIdempotencyOfProcessorStartOperation() throws Exception {
*/
@Test
public void validateStopCallsAreMeaninglessIfProcessorNotStarted() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
final ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -191,6 +192,7 @@ public void validateStopCallsAreMeaninglessIfProcessorNotStarted() throws Except
*/
@Test
public void validateSuccessfullAndOrderlyShutdown() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -232,6 +234,7 @@ public void validateSuccessfullAndOrderlyShutdown() throws Exception {
*/
@Test
public void validateLifecycleOperationOrderWithConcurrentCallsToStartStop() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
final ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -287,6 +290,7 @@ public void run() {
*/
@Test
public void validateProcessorUnscheduledAndStoppedWhenStopIsCalledBeforeProcessorFullyStarted() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -316,11 +320,13 @@ public void validateProcessorUnscheduledAndStoppedWhenStopIsCalledBeforeProcesso
}
/**
- * Validates that Processor is eventually started once invocation
- * of @OnSchedule stopped throwing exceptions.
+ * Validates that Processor is eventually started once invocation of
+ *
+ * @OnSchedule stopped throwing exceptions.
*/
@Test
public void validateProcessScheduledAfterAdministrativeDelayDueToTheOnScheduledException() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -352,6 +358,7 @@ public void validateProcessScheduledAfterAdministrativeDelayDueToTheOnScheduledE
*/
@Test
public void validateProcessorCanBeStoppedWhenOnScheduledConstantlyFails() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -382,7 +389,7 @@ public void validateProcessorCanBeStoppedWhenOnScheduledConstantlyFails() throws
*/
@Test
public void validateProcessorCanBeStoppedWhenOnScheduledBlocksIndefinitelyInterruptable() throws Exception {
- NiFiProperties.getInstance().setProperty(NiFiProperties.PROCESSOR_SCHEDULING_TIMEOUT, "5 sec");
+ this.fc = buildFlowControllerForTest(NiFiProperties.PROCESSOR_SCHEDULING_TIMEOUT, "5 sec");
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -410,7 +417,7 @@ public void validateProcessorCanBeStoppedWhenOnScheduledBlocksIndefinitelyInterr
*/
@Test
public void validateProcessorCanBeStoppedWhenOnScheduledBlocksIndefinitelyUninterruptable() throws Exception {
- NiFiProperties.getInstance().setProperty(NiFiProperties.PROCESSOR_SCHEDULING_TIMEOUT, "5 sec");
+ this.fc = buildFlowControllerForTest(NiFiProperties.PROCESSOR_SCHEDULING_TIMEOUT, "5 sec");
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -443,6 +450,7 @@ public void validateProcessorCanBeStoppedWhenOnScheduledBlocksIndefinitelyUninte
*/
@Test
public void validateProcessorCanBeStoppedWhenOnTriggerThrowsException() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -471,6 +479,7 @@ public void validateProcessorCanBeStoppedWhenOnTriggerThrowsException() throws E
*/
@Test(expected = IllegalStateException.class)
public void validateStartFailsOnInvalidProcessorWithMissingProperty() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
@@ -485,13 +494,13 @@ public void validateStartFailsOnInvalidProcessorWithMissingProperty() throws Exc
*/
@Test(expected = IllegalStateException.class)
public void validateStartFailsOnInvalidProcessorWithDisabledService() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
ControllerServiceNode testServiceNode = fc.createControllerService(TestService.class.getName(), "serv", true);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString());
-
testProcNode.setProperty("P", "hello");
testProcNode.setProperty("S", testServiceNode.getIdentifier());
@@ -508,6 +517,7 @@ public void validateStartFailsOnInvalidProcessorWithDisabledService() throws Exc
*/
@Test
public void validateStartSucceedsOnProcessorWithEnabledService() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
@@ -533,10 +543,12 @@ public void validateStartSucceedsOnProcessorWithEnabledService() throws Exceptio
/**
* Test deletion of processor when connected to another
+ *
* @throws Exception exception
*/
@Test
public void validateProcessorDeletion() throws Exception {
+ fc = this.buildFlowControllerForTest();
ProcessGroup testGroup = fc.createProcessGroup(UUID.randomUUID().toString());
this.setControllerRootGroup(fc, testGroup);
@@ -652,18 +664,25 @@ private void noop(TestProcessor testProcessor) {
testProcessor.setScenario(emptyRunnable, emptyRunnable, emptyRunnable, emptyRunnable);
}
- /**
- *
- */
- private FlowController buildFlowControllerForTest() throws Exception {
- NiFiProperties properties = NiFiProperties.getInstance();
- properties.setProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, MockProvenanceRepository.class.getName());
- properties.setProperty("nifi.remote.input.socket.port", "");
- properties.setProperty("nifi.remote.input.secure", "");
+ private FlowController buildFlowControllerForTest(final String propKey, final String propValue) throws Exception {
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.ADMINISTRATIVE_YIELD_DURATION, "1 sec");
+ addProps.put(NiFiProperties.STATE_MANAGEMENT_CONFIG_FILE, "target/test-classes/state-management.xml");
+ addProps.put(NiFiProperties.STATE_MANAGEMENT_LOCAL_PROVIDER_ID, "local-provider");
+ addProps.put(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, MockProvenanceRepository.class.getName());
+ addProps.put("nifi.remote.input.socket.port", "");
+ addProps.put("nifi.remote.input.secure", "");
+ if (propKey != null && propValue != null) {
+ addProps.put(propKey, propValue);
+ }
+ final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
+ return FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), nifiProperties,
+ mock(Authorizer.class), mock(AuditService.class), null, new VolatileBulletinRepository(),
+ new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths()));
+ }
- return FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), properties,
- mock(Authorizer.class), mock(AuditService.class), null, new VolatileBulletinRepository(),
- new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths()));
+ private FlowController buildFlowControllerForTest() throws Exception {
+ return buildFlowControllerForTest(null, null);
}
/**
@@ -683,6 +702,7 @@ private void setControllerRootGroup(FlowController controller, ProcessGroup proc
/**
*/
public static class TestProcessor extends AbstractProcessor {
+
private Runnable onScheduleCallback;
private Runnable onUnscheduleCallback;
private Runnable onStopCallback;
@@ -750,8 +770,8 @@ public ValidationResult validate(final String subject, final String value, final
.identifiesControllerService(ITestservice.class)
.build();
- return this.withService ? Arrays.asList(new PropertyDescriptor[] { PROP, SERVICE })
- : Arrays.asList(new PropertyDescriptor[] { PROP });
+ return this.withService ? Arrays.asList(new PropertyDescriptor[]{PROP, SERVICE})
+ : Arrays.asList(new PropertyDescriptor[]{PROP});
}
@Override
@@ -778,6 +798,7 @@ public static interface ITestservice extends ControllerService {
/**
*/
private static class EmptyRunnable implements Runnable {
+
@Override
public void run() {
@@ -787,6 +808,7 @@ public void run() {
/**
*/
private static class BlockingInterruptableRunnable implements Runnable {
+
@Override
public void run() {
try {
@@ -800,6 +822,7 @@ public void run() {
/**
*/
private static class BlockingUninterruptableRunnable implements Runnable {
+
@Override
public void run() {
while (true) {
@@ -815,6 +838,7 @@ public void run() {
/**
*/
private static class RandomOrFixedDelayedRunnable implements Runnable {
+
private final int delayLimit;
private final boolean randomDelay;
@@ -823,6 +847,7 @@ public RandomOrFixedDelayedRunnable(int delayLimit, boolean randomDelay) {
this.randomDelay = randomDelay;
}
Random random = new Random();
+
@Override
public void run() {
try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
index e50cc2781ee4..8a12e95ce036 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
@@ -20,7 +20,6 @@
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
@@ -73,6 +72,7 @@
import org.mockito.Mockito;
public class TestStandardProcessScheduler {
+
private StandardProcessScheduler scheduler = null;
private ReportingTaskNode taskNode = null;
private TestReportingTask reportingTask = null;
@@ -80,17 +80,18 @@ public class TestStandardProcessScheduler {
private VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
private FlowController controller;
private ProcessGroup rootGroup;
+ private NiFiProperties nifiProperties;
@Before
public void setup() throws InitializationException {
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
- this.refreshNiFiProperties();
- scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider,variableRegistry);
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestStandardProcessScheduler.class.getResource("/nifi.properties").getFile());
+ this.nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
+ scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, variableRegistry, nifiProperties);
scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class));
reportingTask = new TestReportingTask();
final ReportingInitializationContext config = new StandardReportingInitializationContext(UUID.randomUUID().toString(), "Test", SchedulingStrategy.TIMER_DRIVEN, "5 secs",
- Mockito.mock(ComponentLog.class), null);
+ Mockito.mock(ComponentLog.class), null, nifiProperties);
reportingTask.initialize(config);
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(null, variableRegistry);
@@ -102,10 +103,11 @@ public void setup() throws InitializationException {
}
/**
- * We have run into an issue where a Reporting Task is scheduled to run but throws an Exception
- * from a method with the @OnScheduled annotation. User stops Reporting Task, updates configuration
- * to fix the issue. Reporting Task then finishes running @OnSchedule method and is then scheduled to run.
- * This unit test is intended to verify that we have this resolved.
+ * We have run into an issue where a Reporting Task is scheduled to run but
+ * throws an Exception from a method with the @OnScheduled annotation. User
+ * stops Reporting Task, updates configuration to fix the issue. Reporting
+ * Task then finishes running @OnSchedule method and is then scheduled to
+ * run. This unit test is intended to verify that we have this resolved.
*/
@Test
public void testReportingTaskDoesntKeepRunningAfterStop() throws InterruptedException, InitializationException {
@@ -129,12 +131,13 @@ public void testReportingTaskDoesntKeepRunningAfterStop() throws InterruptedExce
public void testDisableControllerServiceWithProcessorTryingToStartUsingIt() throws InterruptedException {
final Processor proc = new ServiceReferencingProcessor();
- final StandardControllerServiceProvider serviceProvider = new StandardControllerServiceProvider(controller, scheduler, null, Mockito.mock(StateManagerProvider.class),variableRegistry);
+ final StandardControllerServiceProvider serviceProvider =
+ new StandardControllerServiceProvider(controller, scheduler, null, Mockito.mock(StateManagerProvider.class), variableRegistry, nifiProperties);
final ControllerServiceNode service = serviceProvider.createControllerService(NoStartServiceImpl.class.getName(), "service", true);
rootGroup.addControllerService(service);
final ProcessorNode procNode = new StandardProcessorNode(proc, UUID.randomUUID().toString(),
- new StandardValidationContextFactory(serviceProvider, variableRegistry), scheduler, serviceProvider);
+ new StandardValidationContextFactory(serviceProvider, variableRegistry), scheduler, serviceProvider, nifiProperties);
rootGroup.addProcessor(procNode);
procNode.setProperty(ServiceReferencingProcessor.SERVICE_DESC.getName(), service.getIdentifier());
@@ -150,12 +153,12 @@ public void testDisableControllerServiceWithProcessorTryingToStartUsingIt() thro
scheduler.disableControllerService(service);
assertTrue(service.getState() == ControllerServiceState.DISABLING);
assertFalse(service.isActive());
- Thread.sleep(1000);
+ Thread.sleep(2000);
assertTrue(service.getState() == ControllerServiceState.DISABLED);
}
-
private class TestReportingTask extends AbstractReportingTask {
+
private final AtomicBoolean failOnScheduled = new AtomicBoolean(true);
private final AtomicInteger onScheduleAttempts = new AtomicInteger(0);
private final AtomicInteger triggerCount = new AtomicInteger(0);
@@ -175,8 +178,8 @@ public void onTrigger(final ReportingContext context) {
}
}
-
private static class ServiceReferencingProcessor extends AbstractProcessor {
+
static final PropertyDescriptor SERVICE_DESC = new PropertyDescriptor.Builder()
.name("service")
.identifiesControllerService(NoStartService.class)
@@ -195,15 +198,6 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
}
}
- private void refreshNiFiProperties() {
- try {
- final Field instanceField = NiFiProperties.class.getDeclaredField("instance");
- instanceField.setAccessible(true);
- instanceField.set(null, null);
- } catch (final Exception e) {
- throw new IllegalStateException(e);
- }
- }
/**
* Validates the atomic nature of ControllerServiceNode.enable() method
* which must only trigger @OnEnabled once, regardless of how many threads
@@ -213,7 +207,7 @@ private void refreshNiFiProperties() {
@Test
public void validateServiceEnablementLogicHappensOnlyOnce() throws Exception {
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider,variableRegistry);
+ final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", false);
assertFalse(serviceNode.isActive());
@@ -252,7 +246,7 @@ public void run() {
@Test
public void validateDisabledServiceCantBeDisabled() throws Exception {
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
+ final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", false);
final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
@@ -290,7 +284,7 @@ public void run() {
@Test
public void validateEnabledServiceCanOnlyBeDisabledOnce() throws Exception {
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider,variableRegistry);
+ final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", false);
final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
@@ -324,7 +318,7 @@ public void run() {
@Test
public void validateDisablingOfTheFailedService() throws Exception {
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
+ final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(FailingService.class.getName(),
"1", false);
scheduler.enableControllerService(serviceNode);
@@ -355,7 +349,7 @@ public void validateDisablingOfTheFailedService() throws Exception {
@Test
public void validateEnabledDisableMultiThread() throws Exception {
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
+ final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ExecutorService executor = Executors.newCachedThreadPool();
for (int i = 0; i < 200; i++) {
final ControllerServiceNode serviceNode = provider
@@ -398,7 +392,7 @@ public void run() {
@Test
public void validateNeverEnablingServiceCanStillBeDisabled() throws Exception {
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider,variableRegistry);
+ final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
"1", false);
final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
@@ -417,13 +411,14 @@ public void validateNeverEnablingServiceCanStillBeDisabled() throws Exception {
/**
* Validates that the service that is currently in ENABLING state can be
- * disabled and that its @OnDisabled operation will be invoked as soon
- * as @OnEnable finishes.
+ * disabled and that its @OnDisabled operation will be invoked as soon as
+ *
+ * @OnEnable finishes.
*/
@Test
public void validateLongEnablingServiceCanStillBeDisabled() throws Exception {
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
+ final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
"1", false);
final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
@@ -446,6 +441,7 @@ public void validateLongEnablingServiceCanStillBeDisabled() throws Exception {
}
public static class FailingService extends AbstractControllerService {
+
@OnEnabled
public void enable(final ConfigurationContext context) {
throw new RuntimeException("intentional");
@@ -453,6 +449,7 @@ public void enable(final ConfigurationContext context) {
}
public static class RandomShortDelayEnablingService extends AbstractControllerService {
+
private final Random random = new Random();
@OnEnabled
@@ -490,6 +487,7 @@ public int disableInvocationCount() {
}
public static class LongEnablingService extends AbstractControllerService {
+
private final AtomicInteger enableCounter = new AtomicInteger();
private final AtomicInteger disableCounter = new AtomicInteger();
@@ -520,6 +518,6 @@ public void setLimit(final long limit) {
}
private ProcessScheduler createScheduler() {
- return new StandardProcessScheduler(null, null, stateMgrProvider, variableRegistry);
+ return new StandardProcessScheduler(null, null, stateMgrProvider, variableRegistry, nifiProperties);
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
index 77a1d8dbcb1d..0e4571a13610 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
@@ -19,7 +19,6 @@
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.controller.StandardFlowServiceTest;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.registry.VariableRegistry;
@@ -36,14 +35,15 @@ public class StandardControllerServiceProviderTest {
private ControllerService proxied;
private ControllerService implementation;
private static VariableRegistry variableRegistry;
+ private static NiFiProperties nifiProperties;
@BeforeClass
public static void setupSuite() throws Exception {
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, StandardFlowServiceTest.class.getResource("/conf/nifi.properties").getFile());
- NiFiProperties properties = NiFiProperties.getInstance();
- NarClassLoaders.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, StandardControllerServiceProviderTest.class.getResource("/conf/nifi.properties").getFile());
+ nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
+ NarClassLoaders.getInstance().init(nifiProperties.getFrameworkWorkingDirectory(), nifiProperties.getExtensionsWorkingDirectory());
ExtensionManager.discoverExtensions(NarClassLoaders.getInstance().getExtensionClassLoaders());
- variableRegistry = new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths());
+ variableRegistry = new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths());
}
@Before
@@ -71,7 +71,7 @@ public void disableClusterProvider() {
@Override
public void onComponentRemoved(String componentId) {
}
- }, variableRegistry);
+ }, variableRegistry, nifiProperties);
ControllerServiceNode node = provider.createControllerService(clazz, id, true);
proxied = node.getProxiedControllerService();
implementation = node.getControllerServiceImplementation();
@@ -96,4 +96,4 @@ public void testCallProxiedInitialized() throws InitializationException {
public void testCallImplementationInitialized() throws InitializationException {
implementation.initialize(null);
}
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
index c35fd5913bbb..34033467dc07 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
@@ -47,12 +47,14 @@
import org.apache.nifi.groups.StandardProcessGroup;
import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.util.NiFiProperties;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
public class TestStandardControllerServiceProvider {
+
private static StateManagerProvider stateManagerProvider = new StateManagerProvider() {
@Override
public StateManager getStateManager(final String componentId) {
@@ -80,11 +82,11 @@ public void onComponentRemoved(final String componentId) {
@BeforeClass
public static void setNiFiProps() {
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestStandardControllerServiceProvider.class.getResource("/conf/nifi.properties").getFile());
}
private StandardProcessScheduler createScheduler() {
- return new StandardProcessScheduler(null, null, stateManagerProvider,variableRegistry);
+ return new StandardProcessScheduler(null, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
}
@Test
@@ -94,7 +96,8 @@ public void testDisableControllerService() {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
+ final StandardControllerServiceProvider provider =
+ new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B", false);
provider.enableControllerService(serviceNode);
@@ -108,7 +111,8 @@ public void testEnableDisableWithReference() {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(group);
final ProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
+ final StandardControllerServiceProvider provider =
+ new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B", false);
final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A", false);
@@ -167,7 +171,8 @@ public void testEnableReferencingServicesGraph(final ProcessScheduler scheduler)
final FlowController controller = Mockito.mock(FlowController.class);
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
+ final StandardControllerServiceProvider provider =
+ new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
// build a graph of controller services with dependencies as such:
//
@@ -212,14 +217,14 @@ public void testEnableReferencingServicesGraph(final ProcessScheduler scheduler)
}
}
-
@Test
public void testOrderingOfServices() {
final ProcessGroup procGroup = new MockProcessGroup();
final FlowController controller = Mockito.mock(FlowController.class);
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, null, null, stateManagerProvider,variableRegistry);
+ final StandardControllerServiceProvider provider =
+ new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false);
final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2", false);
@@ -363,7 +368,7 @@ public void testOrderingOfServices() {
private ProcessorNode createProcessor(final StandardProcessScheduler scheduler, final ControllerServiceProvider serviceProvider) {
final ProcessorNode procNode = new StandardProcessorNode(new DummyProcessor(), UUID.randomUUID().toString(),
- new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider);
+ new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider, NiFiProperties.createBasicNiFiProperties(null, null));
final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, null, variableRegistry);
group.addProcessor(procNode);
@@ -379,7 +384,8 @@ public void testEnableReferencingComponents() {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
final StandardProcessScheduler scheduler = createScheduler();
- final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, null, null, stateManagerProvider,variableRegistry);
+ final StandardControllerServiceProvider provider =
+ new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
final ControllerServiceNode serviceNode = provider.createControllerService(ServiceA.class.getName(), "1", false);
final ProcessorNode procNode = createProcessor(scheduler, provider);
@@ -398,7 +404,8 @@ public void testEnableReferencingComponents() {
public void validateEnableServices() {
StandardProcessScheduler scheduler = createScheduler();
FlowController controller = Mockito.mock(FlowController.class);
- StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
+ StandardControllerServiceProvider provider =
+ new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
ProcessGroup procGroup = new MockProcessGroup();
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
@@ -423,7 +430,7 @@ public void validateEnableServices() {
E.setProperty(ServiceA.OTHER_SERVICE.getName(), "A");
E.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "F");
- provider.enableControllerServices(Arrays.asList(new ControllerServiceNode[] { A, B, C, D, E, F }));
+ provider.enableControllerServices(Arrays.asList(new ControllerServiceNode[]{A, B, C, D, E, F}));
assertTrue(A.isActive());
assertTrue(B.isActive());
@@ -443,7 +450,7 @@ public void validateEnableServices2() {
StandardProcessScheduler scheduler = createScheduler();
FlowController controller = Mockito.mock(FlowController.class);
StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null,
- stateManagerProvider, variableRegistry);
+ stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
ProcessGroup procGroup = new MockProcessGroup();
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
@@ -466,7 +473,7 @@ public void validateEnableServices2() {
F.setProperty(ServiceA.OTHER_SERVICE.getName(), "D");
D.setProperty(ServiceA.OTHER_SERVICE.getName(), "C");
- provider.enableControllerServices(Arrays.asList(new ControllerServiceNode[] { C, F, A, B, D }));
+ provider.enableControllerServices(Arrays.asList(new ControllerServiceNode[]{C, F, A, B, D}));
assertTrue(A.isActive());
assertTrue(B.isActive());
@@ -479,7 +486,8 @@ public void validateEnableServices2() {
public void validateEnableServicesWithDisabledMissingService() {
StandardProcessScheduler scheduler = createScheduler();
FlowController controller = Mockito.mock(FlowController.class);
- StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
+ StandardControllerServiceProvider provider =
+ new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, NiFiProperties.createBasicNiFiProperties(null, null));
ProcessGroup procGroup = new MockProcessGroup();
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
@@ -508,7 +516,7 @@ public void validateEnableServicesWithDisabledMissingService() {
serviceNode7.setProperty(ServiceC.REQ_SERVICE_2.getName(), "3");
provider.enableControllerServices(Arrays.asList(
- new ControllerServiceNode[] { serviceNode1, serviceNode2, serviceNode3, serviceNode4, serviceNode5, serviceNode7}));
+ new ControllerServiceNode[]{serviceNode1, serviceNode2, serviceNode3, serviceNode4, serviceNode5, serviceNode7}));
assertFalse(serviceNode1.isActive());
assertFalse(serviceNode2.isActive());
assertFalse(serviceNode3.isActive());
@@ -518,7 +526,7 @@ public void validateEnableServicesWithDisabledMissingService() {
provider.enableControllerService(serviceNode6);
provider.enableControllerServices(Arrays.asList(
- new ControllerServiceNode[] { serviceNode1, serviceNode2, serviceNode3, serviceNode4, serviceNode5 }));
+ new ControllerServiceNode[]{serviceNode1, serviceNode2, serviceNode3, serviceNode4, serviceNode5}));
assertTrue(serviceNode1.isActive());
assertTrue(serviceNode2.isActive());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
index 8ce97fa88661..f8e5b60addb8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.controller.state.providers.zookeeper;
import java.io.IOException;
@@ -38,8 +37,8 @@
public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
- private StateProvider provider;
- private TestingServer zkServer;
+ private volatile StateProvider provider;
+ private volatile TestingServer zkServer;
private static final Map defaultProperties = new HashMap<>();
@@ -49,7 +48,6 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
defaultProperties.put(ZooKeeperStateProvider.ACCESS_CONTROL, ZooKeeperStateProvider.OPEN_TO_WORLD.getValue());
}
-
@Before
public void setup() throws Exception {
zkServer = new TestingServer(true);
@@ -99,9 +97,11 @@ private ZooKeeperStateProvider createProvider(final Map state = new HashMap<>();
@@ -152,7 +150,6 @@ public void testStateTooLargeExceptionThrownOnSetState() throws InterruptedExcep
}
}
-
@Test(timeout = 20000)
public void testStateTooLargeExceptionThrownOnReplace() throws IOException, InterruptedException {
final Map state = new HashMap<>();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunProcessorTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunProcessorTask.java
index 174e5fbc11ea..90e40a2d499b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunProcessorTask.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunProcessorTask.java
@@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.controller.tasks;
import static org.junit.Assert.assertFalse;
@@ -26,6 +25,7 @@
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.util.NiFiProperties;
import org.junit.Test;
import org.mockito.Mockito;
@@ -33,7 +33,7 @@ public class TestContinuallyRunProcessorTask {
@Test
public void testIsWorkToDo() {
- System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestContinuallyRunProcessorTask.class.getResource("/conf/nifi.properties").getFile());
final ProcessorNode procNode = Mockito.mock(ProcessorNode.class);
Mockito.when(procNode.hasIncomingConnection()).thenReturn(false);
@@ -60,7 +60,6 @@ public void testIsWorkToDo() {
when(selfLoopingConnection.getFlowFileQueue()).thenReturn(nonEmptyQueue);
assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode));
-
// Test with only a non-looping Connection that has no FlowFiles
final Connection emptyConnection = Mockito.mock(Connection.class);
when(emptyConnection.getSource()).thenReturn(Mockito.mock(ProcessorNode.class));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
index 381b54b46db7..827abdd0144a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -34,11 +34,14 @@
import java.io.IOException;
import java.io.InputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.List;
+import org.apache.nifi.util.NiFiProperties;
/**
* THREAD SAFE
@@ -165,7 +168,22 @@ Class>[] getExecutionStack() {
}
}
- public static T createInstance(final String implementationClassName, final Class typeDefinition) throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+ /**
+ * Constructs an instance of the given type using either default no args
+ * constructor or a constructor which takes a NiFiProperties object
+ * (preferred).
+ *
+ * @param type
+ * @param implementationClassName class
+ * @param typeDefinition def
+ * @param nifiProperties props
+ * @return constructed instance
+ * @throws InstantiationException ex
+ * @throws IllegalAccessException ex
+ * @throws ClassNotFoundException ex
+ */
+ public static T createInstance(final String implementationClassName, final Class typeDefinition, final NiFiProperties nifiProperties)
+ throws InstantiationException, IllegalAccessException, ClassNotFoundException {
final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
try {
@@ -181,7 +199,16 @@ public static T createInstance(final String implementationClassName, final C
Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
final Class> desiredClass = rawClass.asSubclass(typeDefinition);
- return typeDefinition.cast(desiredClass.newInstance());
+ if (nifiProperties == null) {
+ return typeDefinition.cast(desiredClass.newInstance());
+ }
+ Constructor> constructor = null;
+ try {
+ constructor = desiredClass.getConstructor(NiFiProperties.class);
+ return typeDefinition.cast(constructor.newInstance(nifiProperties));
+ } catch (final NoSuchMethodException | InvocationTargetException ex) {
+ return typeDefinition.cast(desiredClass.newInstance());
+ }
} finally {
Thread.currentThread().setContextClassLoader(originalClassLoader);
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
index 882c8c6fd408..0fc8f4d84620 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
@@ -21,11 +21,8 @@
import org.junit.BeforeClass;
import org.junit.Test;
-import java.io.BufferedInputStream;
import java.io.File;
-import java.io.FileInputStream;
import java.io.IOException;
-import java.io.InputStream;
import java.net.URISyntaxException;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
@@ -37,6 +34,9 @@
import java.util.Set;
import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
@@ -81,7 +81,7 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
@Test
public void testUnpackNars() {
- NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties");
+ NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", Collections.EMPTY_MAP);
assertEquals("./target/NarUnpacker/lib/",
properties.getProperty("nifi.nar.library.directory"));
@@ -112,14 +112,14 @@ public void testUnpackNars() {
@Test
public void testUnpackNarsFromEmptyDir() throws IOException {
- NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties");
-
final File emptyDir = new File("./target/empty/dir");
emptyDir.delete();
emptyDir.deleteOnExit();
assertTrue(emptyDir.mkdirs());
- properties.setProperty("nifi.nar.library.directory.alt", emptyDir.toString());
+ final Map others = new HashMap<>();
+ others.put("nifi.nar.library.directory.alt", emptyDir.toString());
+ NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
@@ -141,8 +141,9 @@ public void testUnpackNarsFromNonExistantDir() {
nonExistantDir.delete();
nonExistantDir.deleteOnExit();
- NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties");
- properties.setProperty("nifi.nar.library.directory.alt", nonExistantDir.toString());
+ final Map others = new HashMap<>();
+ others.put("nifi.nar.library.directory.alt", nonExistantDir.toString());
+ NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
@@ -165,15 +166,16 @@ public void testUnpackNarsFromNonDir() throws IOException {
nonDir.createNewFile();
nonDir.deleteOnExit();
- NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties");
- properties.setProperty("nifi.nar.library.directory.alt", nonDir.toString());
+ final Map others = new HashMap<>();
+ others.put("nifi.nar.library.directory.alt", nonDir.toString());
+ NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
assertNull(extensionMapping);
}
- private NiFiProperties loadSpecifiedProperties(String propertiesFile) {
+ private NiFiProperties loadSpecifiedProperties(final String propertiesFile, final Map others) {
String filePath;
try {
filePath = NarUnpackerTest.class.getResource(propertiesFile).toURI().getPath();
@@ -181,34 +183,6 @@ private NiFiProperties loadSpecifiedProperties(String propertiesFile) {
throw new RuntimeException("Cannot load properties file due to "
+ ex.getLocalizedMessage(), ex);
}
- System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, filePath);
-
- NiFiProperties properties = NiFiProperties.getInstance();
-
- // clear out existing properties
- for (String prop : properties.stringPropertyNames()) {
- properties.remove(prop);
- }
-
- InputStream inStream = null;
- try {
- inStream = new BufferedInputStream(new FileInputStream(filePath));
- properties.load(inStream);
- } catch (final Exception ex) {
- throw new RuntimeException("Cannot load properties file due to "
- + ex.getLocalizedMessage(), ex);
- } finally {
- if (null != inStream) {
- try {
- inStream.close();
- } catch (final Exception ex) {
- /**
- * do nothing *
- */
- }
- }
- }
-
- return properties;
+ return NiFiProperties.createBasicNiFiProperties(filePath, others);
}
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
index a426c40e4d82..6d0fa97c95bb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
@@ -224,7 +224,7 @@ public void run() {
public static void main(String[] args) {
logger.info("Launching NiFi...");
try {
- new NiFi(NiFiProperties.getInstance());
+ new NiFi(NiFiProperties.createBasicNiFiProperties(null, null));
} catch (final Throwable t) {
logger.error("Failure to launch NiFi due to " + t, t);
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslServerSocketFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslServerSocketFactory.java
index 59ea312b4307..458157cb8e00 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslServerSocketFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslServerSocketFactory.java
@@ -36,8 +36,8 @@ public class SslServerSocketFactory extends SSLServerSocketFactory {
private SSLServerSocketFactory sslServerSocketFactory;
- public SslServerSocketFactory() {
- final SSLContext sslCtx = SslContextFactory.createSslContext(NiFiProperties.getInstance());
+ public SslServerSocketFactory(final NiFiProperties nifiProperties) {
+ final SSLContext sslCtx = SslContextFactory.createSslContext(nifiProperties);
if (sslCtx == null) {
try {
sslServerSocketFactory = SSLContext.getDefault().getServerSocketFactory();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslSocketFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslSocketFactory.java
index da0e7fbcf8fc..fa6de5657971 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslSocketFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslSocketFactory.java
@@ -36,8 +36,8 @@ public class SslSocketFactory extends SSLSocketFactory {
private final SSLSocketFactory sslSocketFactory;
- public SslSocketFactory() {
- final SSLContext sslCtx = SslContextFactory.createSslContext(NiFiProperties.getInstance());
+ public SslSocketFactory(final NiFiProperties nifiProperties) {
+ final SSLContext sslCtx = SslContextFactory.createSslContext(nifiProperties);
if (sslCtx == null) {
try {
sslSocketFactory = SSLContext.getDefault().getSocketFactory();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java
index b335f4810335..7a001abc2bce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java
@@ -51,7 +51,7 @@ public class HttpRemoteSiteListener implements RemoteSiteListener {
private ProcessGroup rootGroup;
private ScheduledFuture> transactionMaintenanceTask;
- private HttpRemoteSiteListener() {
+ private HttpRemoteSiteListener(final NiFiProperties nifiProperties) {
super();
taskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
@@ -65,10 +65,9 @@ public Thread newThread(final Runnable r) {
}
});
- NiFiProperties properties = NiFiProperties.getInstance();
int txTtlSec;
try {
- final String snapshotFrequency = properties.getProperty(SITE_TO_SITE_HTTP_TRANSACTION_TTL, DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL);
+ final String snapshotFrequency = nifiProperties.getProperty(SITE_TO_SITE_HTTP_TRANSACTION_TTL, DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL);
txTtlSec = (int) FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.SECONDS);
} catch (final Exception e) {
txTtlSec = (int) FormatUtils.getTimeDuration(DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL, TimeUnit.SECONDS);
@@ -78,11 +77,11 @@ public Thread newThread(final Runnable r) {
transactionTtlSec = txTtlSec;
}
- public static HttpRemoteSiteListener getInstance() {
+ public static HttpRemoteSiteListener getInstance(final NiFiProperties nifiProperties) {
if (instance == null) {
synchronized (HttpRemoteSiteListener.class) {
if (instance == null) {
- instance = new HttpRemoteSiteListener();
+ instance = new HttpRemoteSiteListener(nifiProperties);
}
}
}
@@ -90,6 +89,7 @@ public static HttpRemoteSiteListener getInstance() {
}
private class TransactionWrapper {
+
private final FlowFileTransaction transaction;
private final HandshakeProperties handshakenProperties;
private long lastCommunicationAt;
@@ -129,7 +129,7 @@ public void start() throws IOException {
try {
Set transactionIds = transactions.keySet().stream().collect(Collectors.toSet());
transactionIds.stream().filter(tid -> !isTransactionActive(tid))
- .forEach(tid -> cancelTransaction(tid));
+ .forEach(tid -> cancelTransaction(tid));
} catch (Exception e) {
// Swallow exception so that this thread can keep working.
logger.error("An exception occurred while maintaining transactions", e);
@@ -146,7 +146,7 @@ public void cancelTransaction(String transactionId) {
} else {
logger.debug("Cancel a transaction. transactionId={}", transactionId);
FlowFileTransaction t = wrapper.transaction;
- if(t != null && t.getSession() != null){
+ if (t != null && t.getSession() != null) {
logger.info("Cancel a transaction, rollback its session. transactionId={}", transactionId);
try {
t.getSession().rollback();
@@ -158,10 +158,9 @@ public void cancelTransaction(String transactionId) {
}
}
-
@Override
public void stop() {
- if(transactionMaintenanceTask != null) {
+ if (transactionMaintenanceTask != null) {
logger.debug("Stopping transactionMaintenanceTask...");
transactionMaintenanceTask.cancel(true);
}
@@ -191,10 +190,9 @@ private boolean isTransactionActive(TransactionWrapper transaction) {
/**
* @param transactionId transactionId to check
- * @return Returns a HandshakeProperties instance which is created when this transaction is started,
- only if the transaction is active,
- and it holds a HandshakeProperties,
- otherwise return null
+ * @return Returns a HandshakeProperties instance which is created when this
+ * transaction is started, only if the transaction is active, and it holds a
+ * HandshakeProperties, otherwise return null
*/
public HandshakeProperties getHandshakenProperties(final String transactionId) {
TransactionWrapper transaction = transactions.get(transactionId);
@@ -205,7 +203,7 @@ public HandshakeProperties getHandshakenProperties(final String transactionId) {
}
public void holdTransaction(final String transactionId, final FlowFileTransaction transaction,
- final HandshakeProperties handshakenProperties) throws IllegalStateException {
+ final HandshakeProperties handshakenProperties) throws IllegalStateException {
// We don't check expiration of the transaction here, to support large file transport or slow network.
// The availability of current transaction is already checked when the HTTP request was received at SiteToSiteResource.
TransactionWrapper currentTransaction = transactions.remove(transactionId);
@@ -224,7 +222,7 @@ public void holdTransaction(final String transactionId, final FlowFileTransactio
}
public FlowFileTransaction finalizeTransaction(final String transactionId) throws IllegalStateException {
- if (!isTransactionActive(transactionId)){
+ if (!isTransactionActive(transactionId)) {
throw new IllegalStateException("Transaction was not found or not active anymore. transactionId=" + transactionId);
}
TransactionWrapper transaction = transactions.remove(transactionId);
@@ -239,7 +237,7 @@ public FlowFileTransaction finalizeTransaction(final String transactionId) throw
}
public void extendTransaction(final String transactionId) throws IllegalStateException {
- if (!isTransactionActive(transactionId)){
+ if (!isTransactionActive(transactionId)) {
throw new IllegalStateException("Transaction was not found or not active anymore. transactionId=" + transactionId);
}
TransactionWrapper transaction = transactions.get(transactionId);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
index b0ce357a0299..0e29aefb049c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
@@ -122,8 +122,8 @@ public static T receiveServerProtocolNegotiation(fina
}
public static T
- receiveResourceNegotiation(final Class cls, final DataInputStream dis, final DataOutputStream dos, final Class>[] constructorArgClasses, final Object[] constructorArgs)
- throws IOException, HandshakeException {
+ receiveResourceNegotiation(final Class cls, final DataInputStream dis, final DataOutputStream dos, final Class>[] constructorArgClasses, final Object[] constructorArgs)
+ throws IOException, HandshakeException {
final String resourceClassName = dis.readUTF();
final T resource;
try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
index bd9d2049b4aa..9f9e3aaa2a96 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
@@ -46,6 +46,8 @@
import java.util.Optional;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.util.NiFiProperties;
public class SocketRemoteSiteListener implements RemoteSiteListener {
@@ -55,18 +57,20 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
private final SSLContext sslContext;
private final NodeInformant nodeInformant;
private final AtomicReference rootGroup = new AtomicReference<>();
+ private final NiFiProperties nifiProperties;
private final AtomicBoolean stopped = new AtomicBoolean(false);
private static final Logger LOG = LoggerFactory.getLogger(SocketRemoteSiteListener.class);
- public SocketRemoteSiteListener(final int socketPort, final SSLContext sslContext) {
- this(socketPort, sslContext, null);
+ public SocketRemoteSiteListener(final int socketPort, final SSLContext sslContext, final NiFiProperties nifiProperties) {
+ this(socketPort, sslContext, nifiProperties, null);
}
- public SocketRemoteSiteListener(final int socketPort, final SSLContext sslContext, final NodeInformant nodeInformant) {
+ public SocketRemoteSiteListener(final int socketPort, final SSLContext sslContext, final NiFiProperties nifiProperties, final NodeInformant nodeInformant) {
this.socketPort = socketPort;
this.sslContext = sslContext;
+ this.nifiProperties = nifiProperties;
this.nodeInformant = nodeInformant;
}
@@ -267,7 +271,14 @@ public void run() {
protocol.getPort().receiveFlowFiles(peer, protocol);
break;
case REQUEST_PEER_LIST:
- protocol.sendPeerList(peer, nodeInformant == null ? Optional.empty() : Optional.of(nodeInformant.getNodeInformation()));
+ final Optional nodeInfo = (nodeInformant == null) ? Optional.empty() : Optional.of(nodeInformant.getNodeInformation());
+ protocol.sendPeerList(
+ peer,
+ nodeInfo,
+ nifiProperties.getRemoteInputHost(),
+ nifiProperties.getRemoteInputPort(),
+ nifiProperties.getRemoteInputHttpPort(),
+ nifiProperties.isSiteToSiteSecure());
break;
case SHUTDOWN:
protocol.shutdown(peer);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index 8f115f7b1960..1996357c53ee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -74,14 +74,17 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
private final AtomicBoolean targetRunning = new AtomicBoolean(true);
private final SSLContext sslContext;
private final TransferDirection transferDirection;
+ private final NiFiProperties nifiProperties;
private final AtomicReference clientRef = new AtomicReference<>();
+
SiteToSiteClient getSiteToSiteClient() {
return clientRef.get();
}
public StandardRemoteGroupPort(final String id, final String name, final ProcessGroup processGroup, final RemoteProcessGroup remoteGroup,
- final TransferDirection direction, final ConnectableType type, final SSLContext sslContext, final ProcessScheduler scheduler) {
+ final TransferDirection direction, final ConnectableType type, final SSLContext sslContext, final ProcessScheduler scheduler,
+ final NiFiProperties nifiProperties) {
// remote group port id needs to be unique but cannot just be the id of the port
// in the remote group instance. this supports referencing the same remote
// instance more than once.
@@ -90,11 +93,12 @@ public StandardRemoteGroupPort(final String id, final String name, final Process
this.remoteGroup = remoteGroup;
this.transferDirection = direction;
this.sslContext = sslContext;
+ this.nifiProperties = nifiProperties;
setScheduldingPeriod(MINIMUM_SCHEDULING_NANOS + " nanos");
}
- private static File getPeerPersistenceFile(final String portId) {
- final File stateDir = NiFiProperties.getInstance().getPersistentStateDirectory();
+ private static File getPeerPersistenceFile(final String portId, final NiFiProperties nifiProperties) {
+ final File stateDir = nifiProperties.getPersistentStateDirectory();
return new File(stateDir, portId + ".peers");
}
@@ -138,17 +142,17 @@ public void onSchedulingStart() {
final long penalizationMillis = FormatUtils.getTimeDuration(remoteGroup.getYieldDuration(), TimeUnit.MILLISECONDS);
final SiteToSiteClient client = new SiteToSiteClient.Builder()
- .url(remoteGroup.getTargetUri().toString())
- .portIdentifier(getIdentifier())
- .sslContext(sslContext)
- .useCompression(isUseCompression())
- .eventReporter(remoteGroup.getEventReporter())
- .peerPersistenceFile(getPeerPersistenceFile(getIdentifier()))
- .nodePenalizationPeriod(penalizationMillis, TimeUnit.MILLISECONDS)
- .timeout(remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
- .transportProtocol(remoteGroup.getTransportProtocol())
- .httpProxy(new HttpProxy(remoteGroup.getProxyHost(), remoteGroup.getProxyPort(), remoteGroup.getProxyUser(), remoteGroup.getProxyPassword()))
- .build();
+ .url(remoteGroup.getTargetUri().toString())
+ .portIdentifier(getIdentifier())
+ .sslContext(sslContext)
+ .useCompression(isUseCompression())
+ .eventReporter(remoteGroup.getEventReporter())
+ .peerPersistenceFile(getPeerPersistenceFile(getIdentifier(), nifiProperties))
+ .nodePenalizationPeriod(penalizationMillis, TimeUnit.MILLISECONDS)
+ .timeout(remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
+ .transportProtocol(remoteGroup.getTransportProtocol())
+ .httpProxy(new HttpProxy(remoteGroup.getProxyHost(), remoteGroup.getProxyPort(), remoteGroup.getProxyUser(), remoteGroup.getProxyPassword()))
+ .build();
clientRef.set(client);
}
@@ -306,7 +310,7 @@ public void process(final InputStream in) throws IOException {
final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[]{
- this, flowFileDescription, dataSize, transaction.getCommunicant().getUrl(), uploadMillis, uploadDataRate});
+ this, flowFileDescription, dataSize, transaction.getCommunicant().getUrl(), uploadMillis, uploadDataRate});
return flowFilesSent.size();
} catch (final Exception e) {
@@ -364,7 +368,7 @@ private int receiveFlowFiles(final Transaction transaction, final ProcessContext
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesReceived);
logger.info("{} Successfully receveied {} ({}) from {} in {} milliseconds at a rate of {}", new Object[]{
- this, flowFileDescription, dataSize, transaction.getCommunicant().getUrl(), uploadMillis, uploadDataRate});
+ this, flowFileDescription, dataSize, transaction.getCommunicant().getUrl(), uploadMillis, uploadDataRate});
}
return flowFilesReceived.size();
@@ -386,16 +390,16 @@ public Collection getValidationErrors() {
ValidationResult error = null;
if (!targetExists.get()) {
error = new ValidationResult.Builder()
- .explanation(String.format("Remote instance indicates that port '%s' no longer exists.", getName()))
- .subject(String.format("Remote port '%s'", getName()))
- .valid(false)
- .build();
+ .explanation(String.format("Remote instance indicates that port '%s' no longer exists.", getName()))
+ .subject(String.format("Remote port '%s'", getName()))
+ .valid(false)
+ .build();
} else if (getConnectableType() == ConnectableType.REMOTE_OUTPUT_PORT && getConnections(Relationship.ANONYMOUS).isEmpty()) {
error = new ValidationResult.Builder()
- .explanation(String.format("Port '%s' has no outbound connections", getName()))
- .subject(String.format("Remote port '%s'", getName()))
- .valid(false)
- .build();
+ .explanation(String.format("Port '%s' has no outbound connections", getName()))
+ .subject(String.format("Remote port '%s'", getName()))
+ .valid(false)
+ .build();
}
if (error != null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java
index 860036827fbe..fe324add1c74 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java
@@ -201,7 +201,7 @@ public final void handshake(final Peer peer) throws IOException, HandshakeExcept
handshakeCompleted = true;
}
- abstract protected HandshakeProperties doHandshake(final Peer peer) throws IOException, HandshakeException;
+ abstract protected HandshakeProperties doHandshake(final Peer peer) throws IOException, HandshakeException;
@Override
public int transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
@@ -394,9 +394,10 @@ protected Response readTransactionResponse(boolean isTransfer, CommunicationsSes
protected final void writeTransactionResponse(boolean isTransfer, ResponseCode response, CommunicationsSession commsSession) throws IOException {
writeTransactionResponse(isTransfer, response, commsSession, null);
}
+
protected void writeTransactionResponse(boolean isTransfer, ResponseCode response, CommunicationsSession commsSession, String explanation) throws IOException {
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
- if(explanation == null){
+ if (explanation == null) {
response.writeResponse(dos);
} else {
response.writeResponse(dos, explanation);
@@ -436,7 +437,7 @@ public int receiveFlowFiles(final Peer peer, final ProcessContext context, final
final CheckedInputStream checkedInputStream = new CheckedInputStream(flowFileInputStream, crc);
final DataPacket dataPacket = codec.decode(checkedInputStream);
- if(dataPacket == null){
+ if (dataPacket == null) {
logger.debug("{} Received null dataPacket indicating the end of transaction from {}", this, peer);
break;
}
@@ -528,7 +529,7 @@ protected int commitReceiveTransaction(Peer peer, FlowFileTransaction transactio
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesReceived);
logger.info("{} Successfully received {} ({}) from {} in {} milliseconds at a rate of {}", new Object[]{
- this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+ this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
return flowFilesReceived.size();
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/FlowFileTransaction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/FlowFileTransaction.java
index 560cbaffb591..4f14fbbed079 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/FlowFileTransaction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/FlowFileTransaction.java
@@ -33,7 +33,7 @@ public class FlowFileTransaction {
private final String calculatedCRC;
public FlowFileTransaction() {
- this(null, null, new StopWatch(true), 0, null, null);
+ this(null, null, new StopWatch(true), 0, null, null);
}
public FlowFileTransaction(ProcessSession session, ProcessContext context, StopWatch stopWatch, long bytesSent, Set flowFilesSent, String calculatedCRC) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/HandshakeProperties.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/HandshakeProperties.java
index c4538da6930a..b12a5b5e6f5a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/HandshakeProperties.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/HandshakeProperties.java
@@ -28,7 +28,6 @@ public class HandshakeProperties {
private long batchBytes = 0L;
private long batchDurationNanos = 0L;
-
public String getCommsIdentifier() {
return commsIdentifier;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java
index b2171df0298e..e71ac1d9525b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java
@@ -40,6 +40,7 @@
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Optional;
+import org.apache.nifi.util.NiFiProperties;
public class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerProtocol implements HttpFlowFileServerProtocol {
@@ -47,11 +48,12 @@ public class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerPr
private final FlowFileCodec codec = new StandardFlowFileCodec();
private final VersionNegotiator versionNegotiator;
- private final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ private final HttpRemoteSiteListener transactionManager;
- public StandardHttpFlowFileServerProtocol(final VersionNegotiator versionNegotiator) {
+ public StandardHttpFlowFileServerProtocol(final VersionNegotiator versionNegotiator, final NiFiProperties nifiProperties) {
super();
this.versionNegotiator = versionNegotiator;
+ this.transactionManager = HttpRemoteSiteListener.getInstance(nifiProperties);
}
@Override
@@ -91,7 +93,7 @@ protected void writeTransactionResponse(boolean isTransfer, ResponseCode respons
HttpServerCommunicationsSession commSession = (HttpServerCommunicationsSession) commsSession;
commSession.setResponseCode(response);
- if(isTransfer){
+ if (isTransfer) {
switch (response) {
case NO_MORE_DATA:
logger.debug("{} There's no data to send.", this);
@@ -136,8 +138,8 @@ protected Response readTransactionResponse(boolean isTransfer, CommunicationsSes
ByteArrayOutputStream bos = new ByteArrayOutputStream();
Transaction.TransactionState currentStatus = commSession.getStatus();
- if(isTransfer){
- switch (currentStatus){
+ if (isTransfer) {
+ switch (currentStatus) {
case DATA_EXCHANGED:
String clientChecksum = commSession.getChecksum();
logger.debug("readTransactionResponse. clientChecksum={}", clientChecksum);
@@ -149,7 +151,7 @@ protected Response readTransactionResponse(boolean isTransfer, CommunicationsSes
break;
}
} else {
- switch (currentStatus){
+ switch (currentStatus) {
case TRANSACTION_STARTED:
logger.debug("readTransactionResponse. returning CONTINUE_TRANSACTION.");
// We don't know if there's more data to receive, so just continue it.
@@ -159,7 +161,7 @@ protected Response readTransactionResponse(boolean isTransfer, CommunicationsSes
// Checksum was successfully validated at client side, or BAD_CHECKSUM is returned.
ResponseCode responseCode = commSession.getResponseCode();
logger.debug("readTransactionResponse. responseCode={}", responseCode);
- if(responseCode.containsMessage()){
+ if (responseCode.containsMessage()) {
responseCode.writeResponse(new DataOutputStream(bos), "");
} else {
responseCode.writeResponse(new DataOutputStream(bos));
@@ -226,7 +228,8 @@ public VersionNegotiator getVersionNegotiator() {
}
@Override
- public void sendPeerList(final Peer peer, final Optional clusterNodeInformation) throws IOException {
+ public void sendPeerList(Peer peer, Optional clusterNodeInfo, String remoteInputHost,
+ int remoteInputPort, int remoteInputHttpPort, boolean isSiteToSiteSecure) throws IOException {
}
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index 574c726b6649..e965cf47e888 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@ -30,7 +30,6 @@
import org.apache.nifi.remote.protocol.HandshakeProperties;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.ResponseCode;
-import org.apache.nifi.util.NiFiProperties;
import java.io.DataInputStream;
import java.io.DataOutputStream;
@@ -84,7 +83,7 @@ protected HandshakeProperties doHandshake(Peer peer) throws IOException, Handsha
validateHandshakeRequest(confirmed, peer, properties);
} catch (HandshakeException e) {
ResponseCode handshakeResult = e.getResponseCode();
- if(handshakeResult.containsMessage()){
+ if (handshakeResult.containsMessage()) {
handshakeResult.writeResponse(dos, e.getMessage());
} else {
handshakeResult.writeResponse(dos);
@@ -136,7 +135,6 @@ public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, Protoco
}
}
-
@Override
public RequestType getRequestType(final Peer peer) throws IOException {
if (!handshakeCompleted) {
@@ -154,7 +152,13 @@ public RequestType getRequestType(final Peer peer) throws IOException {
}
@Override
- public void sendPeerList(final Peer peer, final Optional clusterNodeInfo) throws IOException {
+ public void sendPeerList(
+ final Peer peer,
+ final Optional clusterNodeInfo,
+ final String remoteInputHost,
+ final int remoteInputPort,
+ final int remoteInputHttpPort,
+ final boolean isSiteToSiteSecure) throws IOException {
if (!handshakeCompleted) {
throw new IllegalStateException("Handshake has not been completed");
}
@@ -166,11 +170,9 @@ public void sendPeerList(final Peer peer, final Optional
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
- final NiFiProperties properties = NiFiProperties.getInstance();
-
- String remoteInputHost = properties.getRemoteInputHost();
- if (remoteInputHost == null) {
- remoteInputHost = InetAddress.getLocalHost().getHostName();
+ String remoteInputHostVal = remoteInputHost;
+ if (remoteInputHostVal == null) {
+ remoteInputHostVal = InetAddress.getLocalHost().getHostName();
}
logger.debug("{} Advertising Remote Input host name {}", this, peer);
@@ -178,8 +180,8 @@ public void sendPeerList(final Peer peer, final Optional
if (clusterNodeInfo.isPresent()) {
nodeInfos = new ArrayList<>(clusterNodeInfo.get().getNodeInformation());
} else {
- final NodeInformation self = new NodeInformation(remoteInputHost, properties.getRemoteInputPort(), properties.getRemoteInputHttpPort(), properties.getRemoteInputHttpPort(),
- properties.isSiteToSiteSecure(), 0);
+ final NodeInformation self = new NodeInformation(remoteInputHostVal, remoteInputPort, remoteInputHttpPort, remoteInputHttpPort,
+ isSiteToSiteSecure, 0);
nodeInfos = Collections.singletonList(self);
}
@@ -212,7 +214,6 @@ public String getResourceName() {
return RESOURCE_NAME;
}
-
@Override
public VersionNegotiator getVersionNegotiator() {
return versionNegotiator;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java
index e4850952f353..58d4d261066b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java
@@ -39,7 +39,7 @@ public static void setup() {
@Test
public void testNormalTransactionProgress() {
- HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
String transactionId = transactionManager.createTransaction();
assertTrue("Transaction should be active.", transactionManager.isTransactionActive(transactionId));
@@ -59,7 +59,7 @@ public void testNormalTransactionProgress() {
@Test
public void testDuplicatedTransactionId() {
- HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
String transactionId = transactionManager.createTransaction();
assertTrue("Transaction should be active.", transactionManager.isTransactionActive(transactionId));
@@ -78,7 +78,7 @@ public void testDuplicatedTransactionId() {
@Test
public void testNoneExistingTransaction() {
- HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
String transactionId = "does-not-exist-1";
assertFalse("Transaction should not be active.", transactionManager.isTransactionActive(transactionId));
@@ -88,8 +88,8 @@ public void testNoneExistingTransaction() {
try {
transactionManager.holdTransaction(transactionId, transaction, null);
} catch (IllegalStateException e) {
- fail("Transaction can be held even if the transaction id is not valid anymore," +
- " in order to support large file or slow network.");
+ fail("Transaction can be held even if the transaction id is not valid anymore,"
+ + " in order to support large file or slow network.");
}
transactionId = "does-not-exist-2";
@@ -100,5 +100,4 @@ public void testNoneExistingTransaction() {
}
}
-
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
index 4209c93ba3ba..b44f1180836c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
@@ -43,6 +43,7 @@
import java.nio.channels.SocketChannel;
import java.util.HashMap;
import java.util.Map;
+import org.apache.nifi.util.NiFiProperties;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
@@ -70,17 +71,18 @@ public class TestStandardRemoteGroupPort {
@BeforeClass
public static void setup() throws Exception {
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
}
private void setupMock(final SiteToSiteTransportProtocol protocol,
- final TransferDirection direction) throws Exception {
+ final TransferDirection direction) throws Exception {
setupMock(protocol, direction, mock(Transaction.class));
}
private void setupMock(final SiteToSiteTransportProtocol protocol,
- final TransferDirection direction,
- final Transaction transaction) throws Exception {
+ final TransferDirection direction,
+ final Transaction transaction) throws Exception {
processGroup = null;
remoteGroup = mock(RemoteProcessGroup.class);
scheduler = null;
@@ -102,7 +104,7 @@ private void setupMock(final SiteToSiteTransportProtocol protocol,
break;
}
port = spy(new StandardRemoteGroupPort(ID, NAME,
- processGroup, remoteGroup, direction, connectableType, null, scheduler));
+ processGroup, remoteGroup, direction, connectableType, null, scheduler, NiFiProperties.createBasicNiFiProperties(null, null)));
doReturn(true).when(remoteGroup).isTransmitting();
doReturn(protocol).when(remoteGroup).getTransportProtocol();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/TestSocketChannelStreams.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/TestSocketChannelStreams.java
index 03f819076071..27a8807a3bf8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/TestSocketChannelStreams.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/TestSocketChannelStreams.java
@@ -76,7 +76,7 @@
// dos.flush();
//
// final EventReporter eventReporter = Mockito.mock(EventReporter.class);
-// final StandardSiteToSiteProtocol proposedProtocol = new StandardSiteToSiteProtocol(commsSession, eventReporter, NiFiProperties.getInstance());
+// final StandardSiteToSiteProtocol proposedProtocol = new StandardSiteToSiteProtocol(commsSession, eventReporter, nifiProperties);
//
// final StandardSiteToSiteProtocol negotiatedProtocol = (StandardSiteToSiteProtocol) RemoteResourceFactory.initiateResourceNegotiation(proposedProtocol, dis, dos);
// System.out.println(negotiatedProtocol);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/ssl/TestSSLSocketChannel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/ssl/TestSSLSocketChannel.java
index 8fe7149161b7..2ead8571c662 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/ssl/TestSSLSocketChannel.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/io/socket/ssl/TestSSLSocketChannel.java
@@ -67,7 +67,7 @@
// public void testSendingToLocalInstance() throws IOException, InterruptedException, HandshakeException, UnknownPortException, PortNotRunningException, URISyntaxException {
// System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
//
-// final NiFiProperties properties = NiFiProperties.getInstance();
+// final NiFiProperties properties;
// final SSLContext sslContext = SslContextFactory.createSslContext(properties);
//
// final SSLSocketChannel channel = new SSLSocketChannel(sslContext, "localhost", 5000, true);
@@ -84,7 +84,7 @@
// dos.flush();
//
// final EventReporter eventReporter = Mockito.mock(EventReporter.class);
-// final StandardSiteToSiteProtocol proposedProtocol = new StandardSiteToSiteProtocol(commsSession, eventReporter, NiFiProperties.getInstance());
+// final StandardSiteToSiteProtocol proposedProtocol = new StandardSiteToSiteProtocol(commsSession, eventReporter, nifiProperties);
// final StandardSiteToSiteProtocol negotiatedProtocol = (StandardSiteToSiteProtocol) RemoteResourceFactory.initiateResourceNegotiation(proposedProtocol, dis, dos);
// System.out.println(negotiatedProtocol);
//
@@ -104,7 +104,7 @@
// public void testWithSimpleSSLSocket() throws IOException, InterruptedException {
// System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
//
-// final NiFiProperties properties = NiFiProperties.getInstance();
+// final NiFiProperties properties;
// final SSLContext sslContext = SslContextFactory.createSslContext(properties);
//
// final ServerThread server = new ServerThread(sslContext);
@@ -138,7 +138,7 @@
// public void testDirectChannelComms() throws IOException, InterruptedException {
// System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
//
-// final NiFiProperties properties = NiFiProperties.getInstance();
+// final NiFiProperties properties;
// final SSLContext sslContext = SslContextFactory.createSslContext(properties);
//
// final ServerThread server = new ServerThread(sslContext);
@@ -193,7 +193,7 @@
// public void testWriteTimesOut() throws IOException, InterruptedException {
// System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
//
-// final NiFiProperties properties = NiFiProperties.getInstance();
+// final NiFiProperties properties;
// final SSLContext sslContext = SslContextFactory.createSslContext(properties);
//
// final ServerThread server = new ServerThread(sslContext);
@@ -238,7 +238,7 @@
// public void testInputOutputStreams() throws IOException, InterruptedException {
// System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
//
-// final NiFiProperties properties = NiFiProperties.getInstance();
+// final NiFiProperties properties;
// final SSLContext sslContext = SslContextFactory.createSslContext(properties);
//
// final ServerThread server = new ServerThread(sslContext);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java
index 7c9d30b61c5b..f5e803d10f49 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java
@@ -90,7 +90,7 @@ private Peer getDefaultPeer(final String transactionId) {
private HttpFlowFileServerProtocol getDefaultHttpFlowFileServerProtocol() {
final StandardVersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1);
- return new StandardHttpFlowFileServerProtocol(versionNegotiator);
+ return new StandardHttpFlowFileServerProtocol(versionNegotiator, NiFiProperties.createBasicNiFiProperties(null, null));
}
@Test
@@ -294,7 +294,7 @@ public void testTransferOneFileBadChecksum() throws Exception {
}
private Peer transferOneFile(final HttpFlowFileServerProtocol serverProtocol, final String transactionId) throws IOException {
- final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final Peer peer = getDefaultPeer(transactionId);
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
final String endpointUri = "https://peer-host:8443/nifi-api/output-ports/port-id/transactions/"
@@ -338,7 +338,7 @@ private Peer transferOneFile(final HttpFlowFileServerProtocol serverProtocol, fi
@Test
public void testTransferTwoFiles() throws Exception {
- final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = "testTransferTwoFiles";
final Peer peer = getDefaultPeer(transactionId);
@@ -470,7 +470,7 @@ public void testReceiveOneFileBadChecksum() throws Exception {
}
private void receiveOneFile(final HttpFlowFileServerProtocol serverProtocol, final String transactionId, final Peer peer) throws IOException {
- final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String endpointUri = "https://peer-host:8443/nifi-api/input-ports/port-id/transactions/"
+ transactionId + "/flow-files";
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
@@ -530,7 +530,7 @@ private void receiveOneFile(final HttpFlowFileServerProtocol serverProtocol, fin
@Test
public void testReceiveTwoFiles() throws Exception {
- final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = "testReceiveTwoFile";
final String endpointUri = "https://peer-host:8443/nifi-api/input-ports/port-id/transactions/"
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java
index 314e331626c3..f4f0bf818b00 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/JettyServerTest.java
@@ -17,13 +17,13 @@
package org.apache.nifi.web.server;
+import java.lang.reflect.InvocationTargetException;
+import java.util.HashMap;
+import java.util.Map;
import org.eclipse.jetty.util.ssl.SslContextFactory;
import org.apache.nifi.util.NiFiProperties;
import org.junit.Test;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
@@ -31,17 +31,17 @@
public class JettyServerTest {
@Test
- public void testConfigureSslContextFactoryWithKeystorePasswordAndKeyPassword() throws InvocationTargetException, NoSuchMethodException, InstantiationException, IllegalAccessException {
+ public void testConfigureSslContextFactoryWithKeystorePasswordAndKeyPassword() {
// Expect that if we set both passwords, KeyStore password is used for KeyStore, Key password is used for Key Manager
String testKeystorePassword = "testKeystorePassword";
String testKeyPassword = "testKeyPassword";
- NiFiProperties nifiProperties = createNifiProperties();
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.SECURITY_KEYSTORE_PASSWD, testKeystorePassword);
+ addProps.put(NiFiProperties.SECURITY_KEY_PASSWD, testKeyPassword);
+ NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
SslContextFactory contextFactory = mock(SslContextFactory.class);
- nifiProperties.setProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD, testKeystorePassword);
- nifiProperties.setProperty(NiFiProperties.SECURITY_KEY_PASSWD, testKeyPassword);
-
JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
verify(contextFactory).setKeyStorePassword(testKeystorePassword);
@@ -53,11 +53,11 @@ public void testConfigureSslContextFactoryWithKeyPassword() throws InvocationTar
// Expect that with no KeyStore password, we will only need to set Key Manager Password
String testKeyPassword = "testKeyPassword";
- NiFiProperties nifiProperties = createNifiProperties();
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.SECURITY_KEY_PASSWD, testKeyPassword);
+ NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
SslContextFactory contextFactory = mock(SslContextFactory.class);
- nifiProperties.setProperty(NiFiProperties.SECURITY_KEY_PASSWD, testKeyPassword);
-
JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
verify(contextFactory).setKeyManagerPassword(testKeyPassword);
@@ -69,20 +69,15 @@ public void testConfigureSslContextFactoryWithKeystorePassword() throws Invocati
// Expect that with no KeyPassword, we use the same one from the KeyStore
String testKeystorePassword = "testKeystorePassword";
- NiFiProperties nifiProperties = createNifiProperties();
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.SECURITY_KEYSTORE_PASSWD, testKeystorePassword);
+ NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
SslContextFactory contextFactory = mock(SslContextFactory.class);
- nifiProperties.setProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD, testKeystorePassword);
-
JettyServer.configureSslContextFactory(contextFactory, nifiProperties);
verify(contextFactory).setKeyStorePassword(testKeystorePassword);
verify(contextFactory).setKeyManagerPassword(testKeystorePassword);
}
- private NiFiProperties createNifiProperties() throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException {
- Constructor constructor = NiFiProperties.class.getDeclaredConstructor();
- constructor.setAccessible(true);
- return constructor.newInstance();
- }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
index d5e7d364975e..38a738c6abd3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
@@ -92,6 +92,7 @@
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_SIZE;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_REQUEST_EXPIRATION;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION;
+import org.apache.nifi.util.NiFiProperties;
/**
* RESTful endpoint for managing a SiteToSite connection.
@@ -108,14 +109,19 @@ public class DataTransferResource extends ApplicationResource {
public static final String CHECK_SUM = "checksum";
public static final String RESPONSE_CODE = "responseCode";
-
private static final String PORT_TYPE_INPUT = "input-ports";
private static final String PORT_TYPE_OUTPUT = "output-ports";
private Authorizer authorizer;
private final ResponseCreator responseCreator = new ResponseCreator();
private final VersionNegotiator transportProtocolVersionNegotiator = new TransportProtocolVersionNegotiator(1);
- private final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ private final HttpRemoteSiteListener transactionManager;
+ private final NiFiProperties nifiProperties;
+
+ public DataTransferResource(final NiFiProperties nifiProperties) {
+ this.nifiProperties = nifiProperties;
+ transactionManager = HttpRemoteSiteListener.getInstance(nifiProperties);
+ }
/**
* Authorizes access to data transfers.
@@ -161,18 +167,17 @@ protected void authorizeDataTransfer(final ResourceType resourceType, final Stri
value = "Create a transaction to the specified output port or input port",
response = TransactionResultEntity.class,
authorizations = {
- @Authorization(value = "Write - /data-transfer/{component-type}/{uuid}", type = "")
+ @Authorization(value = "Write - /data-transfer/{component-type}/{uuid}", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 404, message = "The specified resource could not be found."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
- @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),
- }
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
+ @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),}
)
public Response createPortTransaction(
@ApiParam(
@@ -187,7 +192,6 @@ public Response createPortTransaction(
@Context UriInfo uriInfo,
InputStream inputStream) {
-
if (!PORT_TYPE_INPUT.equals(portType) && !PORT_TYPE_OUTPUT.equals(portType)) {
return responseCreator.wrongPortTypeResponse(portType, portId);
}
@@ -235,18 +239,17 @@ public Response createPortTransaction(
value = "Transfer flow files to the input port",
response = String.class,
authorizations = {
- @Authorization(value = "Write - /data-transfer/input-ports/{uuid}", type = "")
+ @Authorization(value = "Write - /data-transfer/input-ports/{uuid}", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 404, message = "The specified resource could not be found."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
- @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),
- }
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
+ @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),}
)
public Response receiveFlowFiles(
@ApiParam(
@@ -299,26 +302,26 @@ public Response receiveFlowFiles(
}
private HttpFlowFileServerProtocol initiateServerProtocol(final HttpServletRequest req, final Peer peer,
- final Integer transportProtocolVersion) throws IOException {
+ final Integer transportProtocolVersion) throws IOException {
// Switch transaction protocol version based on transport protocol version.
TransportProtocolVersionNegotiator negotiatedTransportProtocolVersion = new TransportProtocolVersionNegotiator(transportProtocolVersion);
VersionNegotiator versionNegotiator = new StandardVersionNegotiator(negotiatedTransportProtocolVersion.getTransactionProtocolVersion());
final String dataTransferUrl = req.getRequestURL().toString();
- ((HttpCommunicationsSession)peer.getCommunicationsSession()).setDataTransferUrl(dataTransferUrl);
+ ((HttpCommunicationsSession) peer.getCommunicationsSession()).setDataTransferUrl(dataTransferUrl);
HttpFlowFileServerProtocol serverProtocol = getHttpFlowFileServerProtocol(versionNegotiator);
- HttpRemoteSiteListener.getInstance().setupServerProtocol(serverProtocol);
+ HttpRemoteSiteListener.getInstance(nifiProperties).setupServerProtocol(serverProtocol);
serverProtocol.handshake(peer);
return serverProtocol;
}
HttpFlowFileServerProtocol getHttpFlowFileServerProtocol(final VersionNegotiator versionNegotiator) {
- return new StandardHttpFlowFileServerProtocol(versionNegotiator);
+ return new StandardHttpFlowFileServerProtocol(versionNegotiator, nifiProperties);
}
private Peer constructPeer(final HttpServletRequest req, final InputStream inputStream,
- final OutputStream outputStream, final String portId, final String transactionId) {
+ final OutputStream outputStream, final String portId, final String transactionId) {
final String clientHostName = req.getRemoteHost();
final int clientPort = req.getRemotePort();
@@ -376,18 +379,17 @@ private Peer constructPeer(final HttpServletRequest req, final InputStream input
value = "Commit or cancel the specified transaction",
response = TransactionResultEntity.class,
authorizations = {
- @Authorization(value = "Write - /data-transfer/output-ports/{uuid}", type = "")
+ @Authorization(value = "Write - /data-transfer/output-ports/{uuid}", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 404, message = "The specified resource could not be found."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
- @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),
- }
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
+ @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),}
)
public Response commitOutputPortTransaction(
@ApiParam(
@@ -474,7 +476,6 @@ public Response commitOutputPortTransaction(
return clusterContext(noCache(setCommonHeaders(Response.ok(entity), transportProtocolVersion, transactionManager))).build();
}
-
@DELETE
@Consumes(MediaType.APPLICATION_OCTET_STREAM)
@Produces(MediaType.APPLICATION_JSON)
@@ -483,18 +484,17 @@ public Response commitOutputPortTransaction(
value = "Commit or cancel the specified transaction",
response = TransactionResultEntity.class,
authorizations = {
- @Authorization(value = "Write - /data-transfer/input-ports/{uuid}", type = "")
+ @Authorization(value = "Write - /data-transfer/input-ports/{uuid}", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 404, message = "The specified resource could not be found."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
- @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),
- }
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
+ @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),}
)
public Response commitInputPortTransaction(
@ApiParam(
@@ -592,7 +592,6 @@ private Response cancelTransaction(String transactionId, TransactionResultEntity
return Response.ok(entity).build();
}
-
@GET
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_OCTET_STREAM)
@@ -601,19 +600,18 @@ private Response cancelTransaction(String transactionId, TransactionResultEntity
value = "Transfer flow files from the output port",
response = StreamingOutput.class,
authorizations = {
- @Authorization(value = "Write - /data-transfer/output-ports/{uuid}", type = "")
+ @Authorization(value = "Write - /data-transfer/output-ports/{uuid}", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 200, message = "There is no flow file to return."),
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 404, message = "The specified resource could not be found."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
- @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),
- }
+ @ApiResponse(code = 200, message = "There is no flow file to return."),
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
+ @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),}
)
public Response transferFlowFiles(
@ApiParam(
@@ -685,16 +683,16 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE
value = "Extend transaction TTL",
response = TransactionResultEntity.class,
authorizations = {
- @Authorization(value = "Write - /data-transfer/input-ports/{uuid}", type = "")
+ @Authorization(value = "Write - /data-transfer/input-ports/{uuid}", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 404, message = "The specified resource could not be found."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response extendInputPortTransactionTTL(
@@ -720,18 +718,17 @@ public Response extendInputPortTransactionTTL(
value = "Extend transaction TTL",
response = TransactionResultEntity.class,
authorizations = {
- @Authorization(value = "Write - /data-transfer/output-ports/{uuid}", type = "")
+ @Authorization(value = "Write - /data-transfer/output-ports/{uuid}", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 404, message = "The specified resource could not be found."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
- @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),
- }
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful."),
+ @ApiResponse(code = 503, message = "NiFi instance is not ready for serving request, or temporarily overloaded. Retrying the same request later may be successful"),}
)
public Response extendOutputPortTransactionTTL(
@PathParam("portId") String portId,
@@ -794,6 +791,7 @@ public Response extendPortTransactionTTL(
}
private class ValidateRequestResult {
+
private Integer transportProtocolVersion;
private Response errResponse;
}
@@ -824,9 +822,7 @@ private ValidateRequestResult validateResult(HttpServletRequest req, String port
return result;
}
-
// setters
-
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
index 66b115199916..88bdeb65b8e4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
@@ -16,23 +16,9 @@
*/
package org.apache.nifi.web.api;
-import static org.apache.commons.lang3.StringUtils.isEmpty;
import java.net.InetAddress;
import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.GET;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.Path;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
@@ -79,6 +65,7 @@
import java.util.Set;
import static org.apache.commons.lang3.StringUtils.isEmpty;
+import org.apache.nifi.util.NiFiProperties;
/**
* RESTful endpoint for managing a SiteToSite connection.
@@ -98,7 +85,11 @@ public class SiteToSiteResource extends ApplicationResource {
private final ResponseCreator responseCreator = new ResponseCreator();
private final VersionNegotiator transportProtocolVersionNegotiator = new TransportProtocolVersionNegotiator(1);
- private final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ private final HttpRemoteSiteListener transactionManager;
+
+ public SiteToSiteResource(final NiFiProperties nifiProperties) {
+ transactionManager = HttpRemoteSiteListener.getInstance(nifiProperties);
+ }
/**
* Authorizes access to Site To Site details.
@@ -135,15 +126,15 @@ protected void authorizeSiteToSite() {
value = "Returns the details about this NiFi necessary to communicate via site to site",
response = ControllerEntity.class,
authorizations = {
- @Authorization(value = "Read - /site-to-site", type = "")
+ @Authorization(value = "Read - /site-to-site", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response getSiteToSiteDetails(@Context HttpServletRequest req) {
@@ -173,7 +164,6 @@ public Response getSiteToSiteDetails(@Context HttpServletRequest req) {
return clusterContext(noCache(Response.ok(entity))).build();
}
-
/**
* Returns the available Peers and its status of this NiFi.
*
@@ -188,15 +178,15 @@ public Response getSiteToSiteDetails(@Context HttpServletRequest req) {
value = "Returns the available Peers and its status of this NiFi",
response = PeersEntity.class,
authorizations = {
- @Authorization(value = "Read - /site-to-site", type = "")
+ @Authorization(value = "Read - /site-to-site", type = "")
}
)
@ApiResponses(
value = {
- @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
- @ApiResponse(code = 401, message = "Client could not be authenticated."),
- @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
- @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response getPeers(@Context HttpServletRequest req) {
@@ -261,7 +251,6 @@ public Response getPeers(@Context HttpServletRequest req) {
}
// setters
-
public void setServiceFacade(final NiFiServiceFacade serviceFacade) {
this.serviceFacade = serviceFacade;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
index f740af78d47c..b6f419fd747b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
@@ -26,6 +26,8 @@
import org.apache.nifi.util.NiFiProperties;
import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
import static org.junit.Assert.assertEquals;
@@ -68,12 +70,12 @@ public AccessControlHelper(final String flowXmlPath, final String overrideAuthor
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, nifiPropertiesFile.getAbsolutePath());
// update the flow.xml property
- NiFiProperties props = NiFiProperties.getInstance();
- props.setProperty(NiFiProperties.FLOW_CONFIGURATION_FILE, flowXmlPath);
-
+ final Map addProps = new HashMap<>();
+ addProps.put(NiFiProperties.FLOW_CONFIGURATION_FILE, flowXmlPath);
if (overrideAuthorizer != null) {
- props.setProperty(NiFiProperties.SECURITY_USER_AUTHORIZER, overrideAuthorizer);
+ addProps.put(NiFiProperties.SECURITY_USER_AUTHORIZER, overrideAuthorizer);
}
+ NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null, addProps);
// load extensions
NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
index 3336669e2835..dc07fdf133a7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
@@ -64,8 +64,9 @@ public static void setup() throws Exception {
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, nifiPropertiesFile.getAbsolutePath());
// update the flow.xml property
- NiFiProperties props = NiFiProperties.getInstance();
- props.setProperty("nifi.flow.configuration.file", FLOW_XML_PATH);
+ final Map addProps = new HashMap<>();
+ addProps.put("nifi.flow.configuration.file", FLOW_XML_PATH);
+ NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null, addProps);
// delete the database directory to avoid issues with re-registration in testRequestAccessUsingToken
FileUtils.deleteDirectory(props.getDatabaseRepositoryPath().toFile());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
index 4c1a4173f1e3..55c31680bdcb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
@@ -18,6 +18,8 @@
import com.sun.jersey.api.client.Client;
import java.io.File;
+import java.net.URISyntaxException;
+import java.net.URL;
import java.util.Collections;
import javax.servlet.ServletContext;
import org.apache.nifi.util.NiFiProperties;
@@ -50,8 +52,15 @@ public class NiFiTestServer {
private WebAppContext webappContext;
public NiFiTestServer(String webappRoot, String contextPath) {
- // load the configuration
- properties = NiFiProperties.getInstance();
+ final URL resource = NiFiTestServer.class.getResource("/site-to-site/nifi.properties");
+ try {
+ final String propertiesFile = resource.toURI().getPath();
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, propertiesFile);
+ // load the configuration
+ properties = NiFiProperties.createBasicNiFiProperties(null, null);
+ } catch (final URISyntaxException ue) {
+ throw new RuntimeException();
+ }
createWebAppContext(webappRoot, contextPath);
createServer();
@@ -171,14 +180,16 @@ public Client getClient() {
}
/**
- * Convenience method to provide access to Spring beans accessible from the web application context.
+ * Convenience method to provide access to Spring beans accessible from the
+ * web application context.
*
* @param target cast
* @param beanName name of the spring bean
* @param clazz class of the spring bean
* @return Spring bean with given name and class type
*
- * @throws ClassCastException if the bean found cannot be cast to the given class type
+ * @throws ClassCastException if the bean found cannot be cast to the given
+ * class type
*/
public T getSpringBean(String beanName, Class clazz) {
ServletContext servletContext = webappContext.getServletHandler().getServletContext();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
index 422dbc3fc9ee..eb87830a8350 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
@@ -178,7 +178,7 @@ public void testExtendTransaction() throws Exception {
final UriInfo uriInfo = mockUriInfo(locationUriStr);
final InputStream inputStream = null;
- final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = transactionManager.createTransaction();
final Response response = resource.extendPortTransactionTTL("input-ports", "port-id", transactionId, req, res, context, uriInfo, inputStream);
@@ -209,7 +209,7 @@ public void testReceiveFlowFiles() throws Exception {
final ServletContext context = null;
final InputStream inputStream = null;
- final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = transactionManager.createTransaction();
final Response response = resource.receiveFlowFiles("port-id", transactionId, req, context, inputStream);
@@ -236,7 +236,7 @@ public void testReceiveZeroFlowFiles() throws Exception {
final ServletContext context = null;
final InputStream inputStream = null;
- final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = transactionManager.createTransaction();
final Response response = resource.receiveFlowFiles("port-id", transactionId, req, context, inputStream);
@@ -255,7 +255,7 @@ public void testCommitInputPortTransaction() throws Exception {
final ServletContext context = null;
final InputStream inputStream = null;
- final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = transactionManager.createTransaction();
final Response response = resource.commitInputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(), "port-id", transactionId, req, context, inputStream);
@@ -278,7 +278,7 @@ public void testTransferFlowFiles() throws Exception {
final HttpServletResponse res = null;
final InputStream inputStream = null;
- final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = transactionManager.createTransaction();
final Response response = resource.transferFlowFiles("port-id", transactionId, req, res, context, inputStream);
@@ -300,7 +300,7 @@ public void testCommitOutputPortTransaction() throws Exception {
final ServletContext context = null;
final InputStream inputStream = null;
- final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = transactionManager.createTransaction();
final Response response = resource.commitOutputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(),
@@ -326,7 +326,7 @@ public void testCommitOutputPortTransactionBadChecksum() throws Exception {
final ServletContext context = null;
final InputStream inputStream = null;
- final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
+ final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
final String transactionId = transactionManager.createTransaction();
final Response response = resource.commitOutputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(),
@@ -342,7 +342,7 @@ public void testCommitOutputPortTransactionBadChecksum() throws Exception {
private DataTransferResource getDataTransferResource() {
final HttpFlowFileServerProtocol serverProtocol = mock(HttpFlowFileServerProtocol.class);
- final DataTransferResource resource = new DataTransferResource() {
+ final DataTransferResource resource = new DataTransferResource(NiFiProperties.createBasicNiFiProperties(null, null)) {
@Override
protected void authorizeDataTransfer(ResourceType resourceType, String identifier) {
}
@@ -352,7 +352,7 @@ HttpFlowFileServerProtocol getHttpFlowFileServerProtocol(VersionNegotiator versi
return serverProtocol;
}
};
- resource.setProperties(NiFiProperties.getInstance());
+ resource.setProperties(NiFiProperties.createBasicNiFiProperties(null, null));
return resource;
}
}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestSiteToSiteResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestSiteToSiteResource.java
index 8b65c3a6aaec..60a7ba9280a4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestSiteToSiteResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestSiteToSiteResource.java
@@ -151,12 +151,12 @@ public void testPeersVersionNegotiationDowngrade() throws Exception {
}
private SiteToSiteResource getSiteToSiteResource(final NiFiServiceFacade serviceFacade) {
- final SiteToSiteResource resource = new SiteToSiteResource() {
+ final SiteToSiteResource resource = new SiteToSiteResource(NiFiProperties.createBasicNiFiProperties(null, null)) {
@Override
protected void authorizeSiteToSite() {
}
};
- resource.setProperties(NiFiProperties.getInstance());
+ resource.setProperties(NiFiProperties.createBasicNiFiProperties(null, null));
resource.setServiceFacade(serviceFacade);
return resource;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy
index 575800c3662d..e3ff000ce5eb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy
@@ -65,7 +65,17 @@ public class OcspCertificateValidatorGroovyTest {
@Before
public void setUp() throws Exception {
- mockProperties = [getProperty: { String propertyName -> return "value_for_${propertyName}" }] as NiFiProperties
+ mockProperties = new NiFiProperties() {
+ @Override
+ String getProperty(String key) {
+ return 'value_for_' + key
+ }
+
+ @Override
+ Set getPropertyKeys() {
+ return ["A", "set"]
+ }
+ }
}
@After
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/NiFiAuthenticationProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/NiFiAuthenticationProviderTest.java
index 0e25747b3e98..eb89c22c9d93 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/NiFiAuthenticationProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/NiFiAuthenticationProviderTest.java
@@ -170,7 +170,7 @@ public void testMapIdentityWithMultipleMatchingPatterns() {
private NiFiProperties getNiFiProperties(final Properties properties) {
final NiFiProperties nifiProperties = Mockito.mock(NiFiProperties.class);
- when(nifiProperties.stringPropertyNames()).thenReturn(properties.stringPropertyNames());
+ when(nifiProperties.getPropertyKeys()).thenReturn(properties.stringPropertyNames());
when(nifiProperties.getProperty(anyString())).then(new Answer() {
@Override
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 93e0703cb8d4..09450c2c0a01 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@ -41,7 +41,6 @@
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.StringUtils;
import javax.net.SocketFactory;
@@ -128,6 +127,9 @@ public String toString() {
private long lastKerberosReloginTime;
protected KerberosProperties kerberosProperties;
protected List properties;
+ private volatile String kerberosServicePrincipal = null;
+ private volatile File kerberosConfigFile = null;
+ private volatile File kerberosServiceKeytab = null;
// variables shared by all threads of this processor
// Hadoop Configuration, Filesystem, and UserGroupInformation (optional)
@@ -147,10 +149,13 @@ protected void init(ProcessorInitializationContext context) {
props.add(kerberosProperties.getKerberosKeytab());
props.add(KERBEROS_RELOGIN_PERIOD);
properties = Collections.unmodifiableList(props);
+ kerberosServicePrincipal = context.getKerberosServicePrincipal();
+ kerberosConfigFile = context.getKerberosConfigurationFile();
+ kerberosServiceKeytab = context.getKerberosServiceKeytab();
}
protected KerberosProperties getKerberosProperties() {
- return KerberosProperties.create(NiFiProperties.getInstance());
+ return new KerberosProperties(kerberosConfigFile);
}
@Override
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java
index 9e2193d201b3..80d3867dbe4f 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java
@@ -67,7 +67,7 @@ public void setup() throws IOException {
// mock properties and return a temporary file for the kerberos configuration
mockedProperties = mock(NiFiProperties.class);
when(mockedProperties.getKerberosConfigurationFile()).thenReturn(temporaryFile);
- kerberosProperties = KerberosProperties.create(mockedProperties);
+ kerberosProperties = new KerberosProperties(temporaryFile);
}
@After
@@ -139,8 +139,7 @@ public void testKerberosOptions() throws Exception {
@Test
public void testKerberosOptionsWithBadKerberosConfigFile() throws Exception {
// invalid since the kerberos configuration was changed to a non-existent file
- when(mockedProperties.getKerberosConfigurationFile()).thenReturn(new File("BAD_KERBEROS_PATH"));
- kerberosProperties = KerberosProperties.create(mockedProperties);
+ kerberosProperties = new KerberosProperties(new File("BAD_KERBEROS_PATH"));
SimpleHadoopProcessor processor = new SimpleHadoopProcessor(kerberosProperties);
TestRunner runner = TestRunners.newTestRunner(processor);
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java
index 582346a98571..13c5436df25c 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java
@@ -50,7 +50,7 @@ public class GetHDFSTest {
public void setup() {
mockNiFiProperties = mock(NiFiProperties.class);
when(mockNiFiProperties.getKerberosConfigurationFile()).thenReturn(null);
- kerberosProperties = KerberosProperties.create(mockNiFiProperties);
+ kerberosProperties = new KerberosProperties(null);
}
@Test
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/PutHDFSTest.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/PutHDFSTest.java
index c8f8fb102e23..72c7da991e72 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/PutHDFSTest.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/PutHDFSTest.java
@@ -73,7 +73,7 @@ public static void setUpClass() throws Exception {
public void setup() {
mockNiFiProperties = mock(NiFiProperties.class);
when(mockNiFiProperties.getKerberosConfigurationFile()).thenReturn(null);
- kerberosProperties = KerberosProperties.create(mockNiFiProperties);
+ kerberosProperties = new KerberosProperties(null);
}
@Test
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestCreateHadoopSequenceFile.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestCreateHadoopSequenceFile.java
index 8164fc042aa5..af5fb3b01795 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestCreateHadoopSequenceFile.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestCreateHadoopSequenceFile.java
@@ -70,7 +70,7 @@ public static void setUpClass() {
public void setUp() {
mockNiFiProperties = mock(NiFiProperties.class);
when(mockNiFiProperties.getKerberosConfigurationFile()).thenReturn(null);
- kerberosProperties = KerberosProperties.create(mockNiFiProperties);
+ kerberosProperties = new KerberosProperties(null);
CreateHadoopSequenceFile proc = new TestableCreateHadoopSequenceFile(kerberosProperties);
controller = TestRunners.newTestRunner(proc);
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestFetchHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestFetchHDFS.java
index e49975b6302a..bbfb0953d8ba 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestFetchHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestFetchHDFS.java
@@ -41,7 +41,7 @@ public class TestFetchHDFS {
public void setup() {
mockNiFiProperties = mock(NiFiProperties.class);
when(mockNiFiProperties.getKerberosConfigurationFile()).thenReturn(null);
- kerberosProperties = KerberosProperties.create(mockNiFiProperties);
+ kerberosProperties = new KerberosProperties(null);
proc = new TestableFetchHDFS(kerberosProperties);
runner = TestRunners.newTestRunner(proc);
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java
index d4204ea1750a..a76557241f07 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java
@@ -68,7 +68,7 @@ public class TestListHDFS {
public void setup() throws InitializationException {
mockNiFiProperties = mock(NiFiProperties.class);
when(mockNiFiProperties.getKerberosConfigurationFile()).thenReturn(null);
- kerberosProperties = KerberosProperties.create(mockNiFiProperties);
+ kerberosProperties = new KerberosProperties(null);
proc = new ListHDFSWithMockedFileSystem(kerberosProperties);
runner = TestRunners.newTestRunner(proc);
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/inotify/TestGetHDFSEvents.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/inotify/TestGetHDFSEvents.java
index f8c1cce4292f..939cbfc51162 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/inotify/TestGetHDFSEvents.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/inotify/TestGetHDFSEvents.java
@@ -57,7 +57,7 @@ public class TestGetHDFSEvents {
public void setup() {
mockNiFiProperties = mock(NiFiProperties.class);
when(mockNiFiProperties.getKerberosConfigurationFile()).thenReturn(null);
- kerberosProperties = KerberosProperties.create(mockNiFiProperties);
+ kerberosProperties = new KerberosProperties(null);
inotifyEventInputStream = mock(DFSInotifyEventInputStream.class);
hdfsAdmin = mock(HdfsAdmin.class);
}
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
index c2ec43b5dff4..0f7900222860 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.dbcp.hive;
+import java.io.File;
import org.apache.commons.dbcp.BasicDataSource;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
@@ -35,7 +36,6 @@
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.hive.AuthenticationFailedException;
import org.apache.nifi.util.hive.HiveConfigurator;
import org.apache.nifi.util.hive.HiveUtils;
@@ -47,14 +47,15 @@
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
+import org.apache.nifi.controller.ControllerServiceInitializationContext;
/**
- * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
+ * Implementation for Database Connection Pooling Service used for Apache Hive
+ * connections. Apache DBCP is used for connection pooling functionality.
*/
@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
@@ -122,9 +123,8 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
private static final long TICKET_RENEWAL_PERIOD = 60000;
private final static List properties;
- private static KerberosProperties kerberosProperties;
- private String connectionUrl = "unknown";
+ private String connectionUrl = "unknown";
// Holder of cached Configuration information so validation does not reload the same config over and over
private final AtomicReference validationResourceHolder = new AtomicReference<>();
@@ -133,21 +133,31 @@ public class HiveConnectionPool extends AbstractControllerService implements Hiv
private volatile HiveConfigurator hiveConfigurator = new HiveConfigurator();
private volatile UserGroupInformation ugi;
+ private volatile String kerberosServicePrincipal = null;
+ private volatile File kerberosConfigFile = null;
+ private volatile File kerberosServiceKeytab = null;
+ private volatile KerberosProperties kerberosProperties;
static {
- kerberosProperties = KerberosProperties.create(NiFiProperties.getInstance());
List props = new ArrayList<>();
props.add(DATABASE_URL);
props.add(HIVE_CONFIGURATION_RESOURCES);
- props.add(kerberosProperties.getKerberosPrincipal());
- props.add(kerberosProperties.getKerberosKeytab());
props.add(DB_USER);
props.add(DB_PASSWORD);
props.add(MAX_WAIT_TIME);
props.add(MAX_TOTAL_CONNECTIONS);
- properties = Collections.unmodifiableList(props);
+ properties = props;
}
+ @Override
+ protected void init(final ControllerServiceInitializationContext context) {
+ kerberosServicePrincipal = context.getKerberosServicePrincipal();
+ kerberosConfigFile = context.getKerberosConfigurationFile();
+ kerberosServiceKeytab = context.getKerberosServiceKeytab();
+ kerberosProperties = new KerberosProperties(kerberosConfigFile);
+ properties.add(kerberosProperties.getKerberosPrincipal());
+ properties.add(kerberosProperties.getKerberosKeytab());
+ }
@Override
protected List getSupportedPropertyDescriptors() {
@@ -213,7 +223,6 @@ public void onConfigured(final ConfigurationContext context) throws Initializati
}
getLogger().info("Successfully logged in as principal {} with keytab {}", new Object[]{principal, keyTab});
-
}
final String user = context.getProperty(DB_USER).getValue();
final String passw = context.getProperty(DB_PASSWORD).getValue();
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
index c14d0aafd3c8..fdff58bcf3ff 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveStreaming.java
@@ -52,8 +52,6 @@
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.StringUtils;
import org.apache.nifi.util.hive.AuthenticationFailedException;
import org.apache.nifi.util.hive.HiveConfigurator;
import org.apache.nifi.util.hive.HiveOptions;
@@ -367,7 +365,7 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro
final List partitionColumnList;
final String partitionColumns = context.getProperty(PARTITION_COLUMNS).getValue();
- if (StringUtils.isEmpty(partitionColumns)) {
+ if (partitionColumns == null || partitionColumns.isEmpty()) {
partitionColumnList = Collections.emptyList();
} else {
String[] partitionCols = partitionColumns.split(",");
@@ -832,7 +830,7 @@ protected HiveWriter makeHiveWriter(HiveEndPoint endPoint, ExecutorService callT
}
protected KerberosProperties getKerberosProperties() {
- return KerberosProperties.create(NiFiProperties.getInstance());
+ return kerberosProperties;
}
protected class HiveStreamingRecord {
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java
index f95a37766e54..f32c30186025 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveStreaming.java
@@ -34,7 +34,6 @@
import org.apache.nifi.hadoop.KerberosProperties;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.apache.nifi.util.hive.HiveOptions;
@@ -60,7 +59,6 @@
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
/**
* Unit tests for PutHiveStreaming processor.
@@ -81,13 +79,9 @@ public void setUp() throws Exception {
System.setProperty("java.security.krb5.realm", "nifi.com");
System.setProperty("java.security.krb5.kdc", "nifi.kdc");
- NiFiProperties niFiPropertiesWithKerberos = mock(NiFiProperties.class);
- when(niFiPropertiesWithKerberos.getKerberosConfigurationFile()).thenReturn(new File("src/test/resources/krb5.conf"));
- kerberosPropsWithFile = KerberosProperties.create(niFiPropertiesWithKerberos);
+ kerberosPropsWithFile = new KerberosProperties(new File("src/test/resources/krb5.conf"));
- NiFiProperties niFiPropertiesWithoutKerberos = mock(NiFiProperties.class);
- when(niFiPropertiesWithKerberos.getKerberosConfigurationFile()).thenReturn(null);
- kerberosPropsWithoutFile = KerberosProperties.create(niFiPropertiesWithoutKerberos);
+ kerberosPropsWithoutFile = new KerberosProperties(null);
processor = new MockPutHiveStreaming();
processor.setKerberosProperties(kerberosPropsWithFile);
@@ -520,6 +514,7 @@ private byte[] createAvroRecord(List