mirror of https://github.com/apache/nifi.git
NIFI-2574 Changed NiFiProperties to avoid static initializer and updated all references to it.
This commit is contained in:
parent
b3f36489ad
commit
7d7401add4
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -18,16 +18,17 @@ package org.apache.nifi.processor;
|
|||
|
||||
import org.apache.nifi.controller.ControllerServiceLookup;
|
||||
import org.apache.nifi.controller.NodeTypeProvider;
|
||||
import org.apache.nifi.kerberos.KerberosContext;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* The <code>ProcessorInitializationContext</code> 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.
|
||||
* </p>
|
||||
*/
|
||||
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();
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.reporting;
|
|||
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 @@ import org.apache.nifi.scheduling.SchedulingStrategy;
|
|||
* 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
|
||||
|
|
|
@ -34,10 +34,6 @@
|
|||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-processor-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-properties</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
|
|
|
@ -23,20 +23,20 @@ import org.apache.nifi.components.ValidationResult;
|
|||
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 class KerberosProperties {
|
|||
.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 class KerberosProperties {
|
|||
// 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 class KerberosProperties {
|
|||
.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 class KerberosProperties {
|
|||
.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");
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.nifi.hadoop;
|
|||
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 org.mockito.Mockito;
|
|||
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 class TestKerberosProperties {
|
|||
|
||||
@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());
|
||||
|
|
|
@ -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.Path;
|
|||
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;
|
||||
import java.util.Set;
|
||||
|
||||
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;
|
||||
/**
|
||||
* 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<String> getPropertyKeys();
|
||||
|
||||
// getters for core properties //
|
||||
public File getFlowConfigurationFile() {
|
||||
|
@ -395,7 +348,8 @@ public class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
List<Path> 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 class NiFiProperties extends Properties {
|
|||
return file;
|
||||
}
|
||||
|
||||
|
||||
// getters for cluster node properties //
|
||||
public boolean isNode() {
|
||||
return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE));
|
||||
|
@ -719,7 +677,6 @@ public class NiFiProperties extends Properties {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public boolean isClustered() {
|
||||
return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE));
|
||||
}
|
||||
|
@ -779,7 +736,8 @@ public class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
final Map<String, Path> 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 class NiFiProperties extends Properties {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class NiFiProperties extends Properties {
|
|||
final Map<String, Path> 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 class NiFiProperties extends Properties {
|
|||
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 class NiFiProperties extends Properties {
|
|||
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 class NiFiProperties extends Properties {
|
|||
final List<Path> vrPropertiesPaths = new ArrayList<>();
|
||||
|
||||
final String vrPropertiesFiles = getVariableRegistryProperties();
|
||||
if(!StringUtils.isEmpty(vrPropertiesFiles)) {
|
||||
if (!StringUtils.isEmpty(vrPropertiesFiles)) {
|
||||
|
||||
final List<String> 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<String, String> additionalProperties) {
|
||||
final Map<String, String> 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<String> getPropertyKeys() {
|
||||
return properties.stringPropertyNames();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,10 +19,7 @@ package org.apache.nifi.util;
|
|||
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 class NiFiPropertiesTest {
|
|||
@Test
|
||||
public void testMissingProperties() {
|
||||
|
||||
NiFiProperties properties = loadSpecifiedProperties("/NiFiProperties/conf/nifi.missing.properties");
|
||||
NiFiProperties properties = loadNiFiProperties("/NiFiProperties/conf/nifi.missing.properties");
|
||||
|
||||
List<Path> directories = properties.getNarLibraryDirectories();
|
||||
|
||||
|
@ -72,7 +69,7 @@ public class NiFiPropertiesTest {
|
|||
@Test
|
||||
public void testBlankProperties() {
|
||||
|
||||
NiFiProperties properties = loadSpecifiedProperties("/NiFiProperties/conf/nifi.blank.properties");
|
||||
NiFiProperties properties = loadNiFiProperties("/NiFiProperties/conf/nifi.blank.properties");
|
||||
|
||||
List<Path> directories = properties.getNarLibraryDirectories();
|
||||
|
||||
|
@ -83,45 +80,14 @@ public class NiFiPropertiesTest {
|
|||
|
||||
}
|
||||
|
||||
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);
|
||||
realPath = NiFiPropertiesTest.class.getResource(propsPath).toURI().getPath();
|
||||
} catch (final URISyntaxException ex) {
|
||||
throw new RuntimeException(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(realPath, null);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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 class MockControllerServiceInitializationContext extends MockControllerSe
|
|||
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.
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 class MockProcessorInitializationContext implements ProcessorInitializati
|
|||
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.
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 class MockReportingInitializationContext extends MockControllerServiceLoo
|
|||
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.
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 class MockControllerServiceInitializationContext implements ControllerSer
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKerberosServicePrincipal() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosServiceKeytab() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosConfigurationFile() {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -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 class MockProcessorInitializationContext implements ProcessorInitializati
|
|||
public NodeTypeProvider getNodeTypeProvider() {
|
||||
return new MockNodeTypeProvider();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKerberosServicePrincipal() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosServiceKeytab() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosConfigurationFile() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 class MockReportingInitializationContext implements ReportingInitializati
|
|||
public ComponentLog getLogger() {
|
||||
return new MockComponentLogger();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKerberosServicePrincipal() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosServiceKeytab() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosConfigurationFile() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.io.File;
|
|||
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 class DocGeneratorTest {
|
|||
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 class DocGeneratorTest {
|
|||
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 @@ public class DocGeneratorTest {
|
|||
}
|
||||
}
|
||||
|
||||
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<String> getPropertyKeys() {
|
||||
return props.stringPropertyNames();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1468,7 +1468,7 @@ public class FileAuthorizerTest {
|
|||
|
||||
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<String>() {
|
||||
@Override
|
||||
|
|
|
@ -43,7 +43,7 @@ public class IdentityMappingUtil {
|
|||
final List<IdentityMapping> 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);
|
||||
|
|
|
@ -48,7 +48,6 @@ import org.apache.nifi.io.socket.SocketConfiguration;
|
|||
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 @@ import org.slf4j.LoggerFactory;
|
|||
*
|
||||
*/
|
||||
public class StandardClusterCoordinationProtocolSender implements ClusterCoordinationProtocolSender {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(StandardClusterCoordinationProtocolSender.class);
|
||||
|
||||
private final ProtocolContext<ProtocolMessage> protocolContext;
|
||||
|
@ -70,10 +70,6 @@ public class StandardClusterCoordinationProtocolSender implements ClusterCoordin
|
|||
private final int maxThreadsPerRequest;
|
||||
private int handshakeTimeoutSeconds;
|
||||
|
||||
public StandardClusterCoordinationProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
|
||||
this(socketConfiguration, protocolContext, NiFiProperties.getInstance().getClusterNodeProtocolThreads());
|
||||
}
|
||||
|
||||
public StandardClusterCoordinationProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext, final int maxThreadsPerRequest) {
|
||||
if (socketConfiguration == null) {
|
||||
throw new IllegalArgumentException("Socket configuration may not be null.");
|
||||
|
@ -90,7 +86,6 @@ public class StandardClusterCoordinationProtocolSender implements ClusterCoordin
|
|||
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 class StandardClusterCoordinationProtocolSender implements ClusterCoordin
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
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) {
|
||||
|
|
|
@ -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.Logger;
|
|||
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 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
|
|||
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 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
|
|||
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 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
|
|||
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 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
|
|||
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 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
|
|||
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<NodeIdentifier, NodeHeartbeat> 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() {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.Collections;
|
|||
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.Logger;
|
|||
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 class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
|
|||
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 class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
|
|||
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 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
|
|||
return new HashSet<>(clusterNodeIds.values());
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException {
|
||||
if (msg.getType() != MessageType.HEARTBEAT) {
|
||||
|
@ -220,7 +218,7 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
|
|||
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);
|
||||
|
||||
|
|
|
@ -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.net.URI;
|
|||
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<EndpointResponseMerger> endpointMergers = new ArrayList<>();
|
||||
static {
|
||||
private final List<EndpointResponseMerger> 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<NodeResponse> nodeResponses) {
|
||||
if (nodeResponses.size() == 1) {
|
||||
|
@ -170,7 +178,6 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
|
|||
return response;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Set<NodeResponse> getProblematicNodeResponses(final Set<NodeResponse> allResponses) {
|
||||
// Check if there are any 2xx responses
|
||||
|
@ -190,7 +197,7 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
|
|||
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 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
|
|||
return allResponses.stream().anyMatch(p -> p.is2xx());
|
||||
}
|
||||
|
||||
|
||||
private void drainResponses(final Set<NodeResponse> 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) {
|
||||
|
|
|
@ -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.List;
|
|||
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.RemoteProcessGroupStatusDescrip
|
|||
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<String, MetricDescriptor<?>> getMetricDescriptors(final URI uri) {
|
||||
|
|
|
@ -75,6 +75,7 @@ import java.util.concurrent.locks.ReadWriteLock;
|
|||
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 class ThreadPoolRequestReplicator implements RequestReplicator {
|
|||
* @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 class ThreadPoolRequestReplicator implements RequestReplicator {
|
|||
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;
|
||||
|
||||
|
|
|
@ -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.ProtocolException;
|
|||
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.Logger;
|
|||
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<ProtocolMessage> protocolContext, final Properties properties) {
|
||||
public CuratorNodeProtocolSender(final SocketConfiguration socketConfig, final ProtocolContext<ProtocolMessage> 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 @@ public class CuratorNodeProtocolSender extends AbstractNodeProtocolSender {
|
|||
|
||||
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 class CuratorNodeProtocolSender extends AbstractNodeProtocolSender {
|
|||
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 <hostname>:<port>", address);
|
||||
+ "that address is %s, but this is not in the expected format of <hostname>:<port>", address);
|
||||
logger.error(message);
|
||||
throw new ProtocolException(message);
|
||||
}
|
||||
|
@ -101,7 +101,7 @@ public class CuratorNodeProtocolSender extends AbstractNodeProtocolSender {
|
|||
}
|
||||
} 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);
|
||||
}
|
||||
|
|
|
@ -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.HashMap;
|
|||
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.controller.cluster.ZooKeeperClientConfig;
|
|||
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.Logger;
|
|||
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<NodeIdentifier, CircularFifoQueue<NodeEvent>> 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 class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 <code>null</code> 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 <code>null</code> value for
|
||||
* currentStatus
|
||||
*
|
||||
* @param nodeId the node id
|
||||
* @param currentStatus the current status, or <code>null</code> if there is no value currently
|
||||
* @param currentStatus the current status, or <code>null</code> if there is
|
||||
* no value currently
|
||||
* @param newStatus the new status to set
|
||||
* @return <code>true</code> if the map was updated, false otherwise
|
||||
*/
|
||||
|
@ -296,7 +301,6 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED, getRoles(nodeId)));
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
|
||||
final Set<NodeIdentifier> connectedNodeIds = getNodeIdentifiers(NodeConnectionState.CONNECTED);
|
||||
|
@ -363,7 +367,6 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
return status == null ? null : status.getState();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Map<NodeConnectionState, List<NodeIdentifier>> getConnectionStates() {
|
||||
final Map<NodeConnectionState, List<NodeIdentifier>> connectionStates = new HashMap<>();
|
||||
|
@ -521,18 +524,18 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
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 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
|
||||
final Set<NodeIdentifier> 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 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
} 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 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
/**
|
||||
* Updates the status of the node with the given ID to the given status and returns <code>true</code>
|
||||
* if successful, <code>false</code> if no node exists with the given ID
|
||||
* Updates the status of the node with the given ID to the given status and
|
||||
* returns <code>true</code> if successful, <code>false</code> if no node
|
||||
* exists with the given ID
|
||||
*
|
||||
* @param status the new status of the node
|
||||
*/
|
||||
|
@ -705,7 +709,8 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
* Notifies other nodes that the status of a node changed
|
||||
*
|
||||
* @param updatedStatus the updated status for a node in the cluster
|
||||
* @param notifyAllNodes if <code>true</code> will notify all nodes. If <code>false</code>, will notify only the cluster coordinator
|
||||
* @param notifyAllNodes if <code>true</code> will notify all nodes. If
|
||||
* <code>false</code>, 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 class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
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 class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
} 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 class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
// 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 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
} 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 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
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 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
// 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<NodeResponse> nodeResponses) {
|
||||
|
@ -1047,7 +1052,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
* state even if they had problems handling the request.
|
||||
*/
|
||||
if (mutableRequest) {
|
||||
final HttpResponseMerger responseMerger = new StandardHttpResponseMerger();
|
||||
final HttpResponseMerger responseMerger = new StandardHttpResponseMerger(nifiProperties);
|
||||
final Set<NodeResponse> problematicNodeResponses = responseMerger.getProblematicNodeResponses(nodeResponses);
|
||||
|
||||
// all nodes failed
|
||||
|
@ -1055,7 +1060,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
|
||||
// some nodes had a problematic response because of a missing counter, ensure the are not disconnected
|
||||
final boolean someNodesFailedMissingCounter = !problematicNodeResponses.isEmpty()
|
||||
&& problematicNodeResponses.size() < nodeResponses.size() && isMissingCounter(problematicNodeResponses, uriPath);
|
||||
&& problematicNodeResponses.size() < nodeResponses.size() && isMissingCounter(problematicNodeResponses, uriPath);
|
||||
|
||||
// ensure nodes stay connected in certain scenarios
|
||||
if (allNodesFailed) {
|
||||
|
@ -1079,12 +1084,15 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
/**
|
||||
* Determines if all problematic responses were due to 404 NOT_FOUND. Assumes that problematicNodeResponses is not empty and is not comprised of responses from all nodes in the cluster (at least
|
||||
* one node contained the counter in question).
|
||||
* Determines if all problematic responses were due to 404 NOT_FOUND.
|
||||
* Assumes that problematicNodeResponses is not empty and is not comprised
|
||||
* of responses from all nodes in the cluster (at least one node contained
|
||||
* the counter in question).
|
||||
*
|
||||
* @param problematicNodeResponses The problematic node responses
|
||||
* @param uriPath The path of the URI for the request
|
||||
* @return Whether all problematic node responses were due to a missing counter
|
||||
* @return Whether all problematic node responses were due to a missing
|
||||
* counter
|
||||
*/
|
||||
private boolean isMissingCounter(final Set<NodeResponse> problematicNodeResponses, final String uriPath) {
|
||||
if (COUNTER_URI_PATTERN.matcher(uriPath).matches()) {
|
||||
|
|
|
@ -33,24 +33,24 @@ import org.springframework.context.ApplicationContextAware;
|
|||
|
||||
public class ThreadPoolRequestReplicatorFactoryBean implements FactoryBean<ThreadPoolRequestReplicator>, 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 class ThreadPoolRequestReplicatorFactoryBean implements FactoryBean<Threa
|
|||
this.applicationContext = applicationContext;
|
||||
}
|
||||
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
public void setProperties(final NiFiProperties properties) {
|
||||
this.nifiProperties = properties;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -27,7 +27,6 @@ import java.util.HashMap;
|
|||
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 @@ public class TestAbstractHeartbeatMonitor {
|
|||
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<String, String> 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 @@ public class TestAbstractHeartbeatMonitor {
|
|||
private Map<NodeIdentifier, NodeHeartbeat> 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
|
||||
|
|
|
@ -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<ClientResponse, Map<String, Integer>> 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<ClientResponse, Object> mockToRequestEntity = [:]
|
||||
|
|
|
@ -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<ClientResponse, Object> mockToRequestEntity = [:]
|
||||
|
|
|
@ -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 class TestThreadPoolRequestReplicator {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class TestThreadPoolRequestReplicator {
|
|||
});
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout = 15000)
|
||||
public void testLongWaitForResponse() {
|
||||
withReplicator(replicator -> {
|
||||
|
@ -132,7 +131,7 @@ public class TestThreadPoolRequestReplicator {
|
|||
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 class TestThreadPoolRequestReplicator {
|
|||
|
||||
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<NodeIdentifier> nodeIds = new HashSet<>();
|
||||
|
@ -167,7 +165,8 @@ public class TestThreadPoolRequestReplicator {
|
|||
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 @@ public class TestThreadPoolRequestReplicator {
|
|||
|
||||
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 class TestThreadPoolRequestReplicator {
|
|||
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<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers,
|
||||
boolean indicateReplicated, boolean verify) {
|
||||
|
@ -278,7 +278,6 @@ public class TestThreadPoolRequestReplicator {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout = 15000)
|
||||
public void testOneNodeRejectsTwoPhaseCommit() {
|
||||
final Set<NodeIdentifier> nodeIds = new HashSet<>();
|
||||
|
@ -287,7 +286,8 @@ public class TestThreadPoolRequestReplicator {
|
|||
|
||||
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 @@ public class TestThreadPoolRequestReplicator {
|
|||
|
||||
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 @@ public class TestThreadPoolRequestReplicator {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
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 @@ public class TestThreadPoolRequestReplicator {
|
|||
}
|
||||
|
||||
private interface WithReplicator {
|
||||
|
||||
void withReplicator(ThreadPoolRequestReplicator replicator) throws Exception;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.io.IOException;
|
|||
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.ProtocolMessage;
|
|||
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.invocation.InvocationOnMock;
|
|||
import org.mockito.stubbing.Answer;
|
||||
|
||||
public class TestNodeClusterCoordinator {
|
||||
|
||||
private NodeClusterCoordinator coordinator;
|
||||
private ClusterCoordinationProtocolSenderListener senderListener;
|
||||
private List<NodeConnectionStatus> nodeStatuses;
|
||||
|
||||
private Properties createProperties() {
|
||||
final Properties props = new Properties();
|
||||
props.put("nifi.zookeeper.connect.string", "localhost:2181");
|
||||
return props;
|
||||
private NiFiProperties createProperties() {
|
||||
final Map<String,String> 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 class TestNodeClusterCoordinator {
|
|||
assertNotNull(statuses);
|
||||
assertEquals(6, statuses.size());
|
||||
final Map<NodeIdentifier, NodeConnectionStatus> 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 class TestNodeClusterCoordinator {
|
|||
assertEquals("Unit Test", statusChange.getDisconnectReason());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testGetConnectionStates() throws IOException {
|
||||
// Add a disconnected node
|
||||
|
@ -316,7 +318,6 @@ public class TestNodeClusterCoordinator {
|
|||
assertTrue(disconnectedIds.contains(createNodeId(1)));
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout = 5000)
|
||||
public void testRequestNodeDisconnect() throws InterruptedException {
|
||||
// Add a connected node
|
||||
|
@ -341,7 +342,6 @@ public class TestNodeClusterCoordinator {
|
|||
assertEquals(NodeConnectionState.DISCONNECTED, status.getState());
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout = 5000)
|
||||
public void testCannotDisconnectLastNode() throws InterruptedException {
|
||||
// Add a connected node
|
||||
|
@ -369,7 +369,6 @@ public class TestNodeClusterCoordinator {
|
|||
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 class TestNodeClusterCoordinator {
|
|||
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 class TestNodeClusterCoordinator {
|
|||
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);
|
||||
|
|
|
@ -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 class Cluster {
|
|||
|
||||
|
||||
public Node createNode() {
|
||||
NiFiProperties.getInstance().setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, getZooKeeperConnectString());
|
||||
NiFiProperties.getInstance().setProperty(NiFiProperties.CLUSTER_IS_NODE, "true");
|
||||
final Map<String, String> 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);
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
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 class Node {
|
|||
|
||||
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<String> getPropertyKeys() {
|
||||
final Set<String> 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.<Revision> emptyList());
|
||||
|
@ -110,7 +127,7 @@ public class Node {
|
|||
|
||||
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 class Node {
|
|||
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);
|
||||
|
|
|
@ -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<ClusterNodeInformation> clusterNodeInfo) throws IOException;
|
||||
void sendPeerList(
|
||||
Peer peer,
|
||||
Optional<ClusterNodeInformation> clusterNodeInfo,
|
||||
String remoteInputHost,
|
||||
int remoteInputPort,
|
||||
int remoteInputHttpPort,
|
||||
boolean isSiteToSiteSecure) throws IOException;
|
||||
|
||||
void shutdown(Peer peer);
|
||||
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.nifi.groups.ProcessGroup;
|
|||
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.concurrent.atomic.AtomicReference;
|
|||
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 @@ public final class StandardConnection implements Connection {
|
|||
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 final class StandardConnection implements Connection {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 final class StandardConnection implements Connection {
|
|||
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 final class StandardConnection implements Connection {
|
|||
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");
|
||||
|
|
|
@ -63,7 +63,8 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* 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
|
||||
* </p>
|
||||
*/
|
||||
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 class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized void initialize(final SwapManagerInitializationContext initializationContext) {
|
||||
this.claimManager = initializationContext.getResourceClaimManager();
|
||||
|
@ -129,7 +135,6 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
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 class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
|
||||
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 class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public List<String> recoverSwapLocations(final FlowFileQueue flowFileQueue) throws IOException {
|
||||
final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() {
|
||||
|
@ -217,13 +220,13 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
|
||||
// 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 class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
|
||||
// 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 class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
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 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
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 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
} catch (final EOFException eof) {
|
||||
final QueueSize queueSize = new QueueSize(numRecords, contentSize);
|
||||
final SwapSummary summary = new StandardSwapSummary(queueSize, maxRecordId, Collections.<ResourceClaim> emptyList());
|
||||
final SwapContents partialContents = new StandardSwapContents(summary, Collections.<FlowFileRecord> emptyList());
|
||||
final SwapSummary summary = new StandardSwapSummary(queueSize, maxRecordId, Collections.<ResourceClaim>emptyList());
|
||||
final SwapContents partialContents = new StandardSwapContents(summary, Collections.<FlowFileRecord>emptyList());
|
||||
throw new IncompleteSwapFileException(swapLocation, partialContents);
|
||||
}
|
||||
|
||||
|
@ -406,7 +409,7 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
|
||||
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<FlowFileRecord> flowFiles = new ArrayList<>(queueSize.getObjectCount());
|
||||
final List<ResourceClaim> resourceClaims = new ArrayList<>(queueSize.getObjectCount());
|
||||
Long maxId = maxRecordId;
|
||||
|
@ -432,7 +435,7 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
|
||||
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 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private void error(final String error) {
|
||||
logger.error(error);
|
||||
if (eventReporter != null) {
|
||||
|
@ -605,9 +607,8 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
private static class SwapFileComparator implements Comparator<String> {
|
||||
|
||||
@Override
|
||||
public int compare(final String o1, final String o2) {
|
||||
if (o1 == o2) {
|
||||
|
|
|
@ -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<RemoteSiteListener> externalSiteListeners = new HashSet<>();
|
||||
private final AtomicReference<CounterRepository> counterRepositoryRef;
|
||||
|
@ -420,7 +420,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
private FlowController(
|
||||
final FlowFileEventRepository flowFileEventRepo,
|
||||
final NiFiProperties properties,
|
||||
final NiFiProperties nifiProperties,
|
||||
final Authorizer authorizer,
|
||||
final AuditService auditService,
|
||||
final StringEncryptor encryptor,
|
||||
|
@ -435,16 +435,16 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
// 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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
// 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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
// 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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* @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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* @throws IllegalArgumentException if <code>uri</code> 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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
return configuredForClustering;
|
||||
}
|
||||
|
||||
|
||||
private void registerForClusterCoordinator() {
|
||||
leaderElectionManager.register(ClusterRoles.CLUSTER_COORDINATOR, new LeaderElectionStateChangeListener() {
|
||||
@Override
|
||||
|
@ -3320,7 +3318,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// 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 class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* 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();
|
||||
|
|
|
@ -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 {
|
||||
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, properties, senderListener, encryptor, true, coordinator, revisionManager, authorizer);
|
||||
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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
|
||||
// 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 class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
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 class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
|
||||
@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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
|
||||
// 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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
|
||||
// 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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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<Template> 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 class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
final List<Template> 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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
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 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
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 class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
}
|
||||
|
||||
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 class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public void loadSnippets(final byte[] bytes) throws IOException {
|
||||
if (bytes.length == 0) {
|
||||
return;
|
||||
|
@ -909,7 +910,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private class SaveReportingTask implements Runnable {
|
||||
|
||||
@Override
|
||||
|
@ -962,6 +962,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
}
|
||||
|
||||
private class SaveHolder {
|
||||
|
||||
private final Calendar saveTime;
|
||||
private final boolean shouldArchive;
|
||||
|
||||
|
|
|
@ -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 class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
// get/create all the reporting task nodes and DTOs, but don't apply their scheduled state yet
|
||||
final Map<ReportingTaskNode,ReportingTaskDTO> reportingTaskNodesToDTOs = new HashMap<>();
|
||||
final Map<ReportingTaskNode, ReportingTaskDTO> 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 class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
.collect(Collectors.toSet());
|
||||
|
||||
// clone the controller services and map the original id to the clone
|
||||
final Map<String,ControllerServiceNode> controllerServiceMapping = new HashMap<>();
|
||||
final Map<String, ControllerServiceNode> controllerServiceMapping = new HashMap<>();
|
||||
for (ControllerServiceNode controllerService : controllerServicesToClone) {
|
||||
final ControllerServiceNode clone = ControllerServiceLoader.cloneControllerService(controller, controllerService);
|
||||
controller.addRootControllerService(clone);
|
||||
|
@ -370,7 +372,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
scaleRootGroup(rootGroup, encodingVersion);
|
||||
|
||||
// now that controller services are loaded and enabled we can apply the scheduled state to each reporting task
|
||||
for (Map.Entry<ReportingTaskNode,ReportingTaskDTO> entry : reportingTaskNodesToDTOs.entrySet()) {
|
||||
for (Map.Entry<ReportingTaskNode, ReportingTaskDTO> entry : reportingTaskNodesToDTOs.entrySet()) {
|
||||
applyReportingTaskScheduleState(controller, entry.getValue(), entry.getKey(), initialized, existingFlowEmpty);
|
||||
}
|
||||
}
|
||||
|
@ -403,15 +405,15 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
}
|
||||
|
||||
private void updateReportingTaskControllerServices(final Set<ReportingTaskNode> reportingTasks, final Map<String,ControllerServiceNode> controllerServiceMapping) {
|
||||
private void updateReportingTaskControllerServices(final Set<ReportingTaskNode> reportingTasks, final Map<String, ControllerServiceNode> controllerServiceMapping) {
|
||||
for (ReportingTaskNode reportingTask : reportingTasks) {
|
||||
if (reportingTask.getProperties() != null) {
|
||||
final Set<Map.Entry<PropertyDescriptor,String>> propertyDescriptors = reportingTask.getProperties().entrySet().stream()
|
||||
final Set<Map.Entry<PropertyDescriptor, String>> propertyDescriptors = reportingTask.getProperties().entrySet().stream()
|
||||
.filter(e -> e.getKey().getControllerServiceDefinition() != null)
|
||||
.filter(e -> controllerServiceMapping.containsKey(e.getValue()))
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
for (Map.Entry<PropertyDescriptor,String> propEntry : propertyDescriptors) {
|
||||
for (Map.Entry<PropertyDescriptor, String> propEntry : propertyDescriptors) {
|
||||
final PropertyDescriptor propertyDescriptor = propEntry.getKey();
|
||||
final ControllerServiceNode clone = controllerServiceMapping.get(propEntry.getValue());
|
||||
reportingTask.setProperty(propertyDescriptor.getName(), clone.getIdentifier());
|
||||
|
@ -490,7 +492,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
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 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
|
||||
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 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
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 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
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 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
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 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
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<Element> labelNodeList = getChildrenByTagName(processGroupElement, "label");
|
||||
for (final Element labelElement : labelNodeList) {
|
||||
|
@ -969,7 +966,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
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 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
|
||||
|
|
|
@ -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 class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
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 class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
* </p>
|
||||
*/
|
||||
private <T> void invokeTaskAsCancelableFuture(final SchedulingAgentCallback callback, final Callable<T> 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);
|
||||
|
|
|
@ -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.curator.retry.RetryNTimes;
|
|||
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 class ClusterProtocolHeartbeater implements Heartbeater {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized void send(final HeartbeatMessage heartbeatMessage) throws IOException {
|
||||
final String heartbeatAddress = getHeartbeatAddress();
|
||||
|
@ -107,7 +106,6 @@ public class ClusterProtocolHeartbeater implements Heartbeater {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (curatorClient != null) {
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.nifi.controller.cluster;
|
|||
|
||||
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 class ZooKeeperClientConfig {
|
|||
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 class ZooKeeperClientConfig {
|
|||
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 class ZooKeeperClientConfig {
|
|||
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) {
|
||||
|
|
|
@ -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.Logger;
|
|||
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<String, LeaderRole> leaderRoles = new HashMap<>();
|
||||
private final Map<String, LeaderElectionStateChangeListener> 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 class CuratorLeaderElectionManager implements LeaderElectionManager {
|
|||
|
||||
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 class CuratorLeaderElectionManager implements LeaderElectionManager {
|
|||
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 class CuratorLeaderElectionManager implements LeaderElectionManager {
|
|||
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 class CuratorLeaderElectionManager implements LeaderElectionManager {
|
|||
return role.isLeader();
|
||||
}
|
||||
|
||||
|
||||
private static class LeaderRole {
|
||||
|
||||
private final LeaderSelector leaderSelector;
|
||||
private final ElectionListener electionListener;
|
||||
|
||||
|
@ -204,8 +193,8 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private class ElectionListener extends LeaderSelectorListenerAdapter implements LeaderSelectorListener {
|
||||
|
||||
private final String roleName;
|
||||
private final LeaderElectionStateChangeListener listener;
|
||||
|
||||
|
|
|
@ -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.logging.ComponentLog;
|
|||
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 class StandardReportingInitializationContext implements ReportingInitiali
|
|||
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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, Path> fileRespositoryPaths = properties.getContentRepositoryPaths();
|
||||
final Map<String, Path> fileRespositoryPaths = nifiProperties.getContentRepositoryPaths();
|
||||
for (final Path path : fileRespositoryPaths.values()) {
|
||||
Files.createDirectories(path);
|
||||
}
|
||||
|
@ -139,21 +155,21 @@ public class FileSystemRepository implements ContentRepository {
|
|||
index = new AtomicLong(0L);
|
||||
|
||||
for (final String containerName : containerNames) {
|
||||
reclaimable.put(containerName, new LinkedBlockingQueue<ResourceClaim>(10000));
|
||||
archivedFiles.put(containerName, new LinkedBlockingQueue<ArchiveInfo>(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 class FileSystemRepository implements ContentRepository {
|
|||
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 class FileSystemRepository implements ContentRepository {
|
|||
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 class FileSystemRepository implements ContentRepository {
|
|||
public void initialize(final ResourceClaimManager claimManager) {
|
||||
this.resourceClaimManager = claimManager;
|
||||
|
||||
final NiFiProperties properties = NiFiProperties.getInstance();
|
||||
|
||||
final Map<String, Path> fileRespositoryPaths = properties.getContentRepositoryPaths();
|
||||
final Map<String, Path> 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<String, Path> containerEntry : containers.entrySet()) {
|
||||
final String containerName = containerEntry.getKey();
|
||||
|
@ -562,7 +576,6 @@ public class FileSystemRepository implements ContentRepository {
|
|||
return resourceClaimManager.incrementClaimantCount(resourceClaim, newClaim);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int getClaimantCount(final ContentClaim claim) {
|
||||
if (claim == null) {
|
||||
|
@ -619,14 +632,13 @@ public class FileSystemRepository implements ContentRepository {
|
|||
|
||||
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 class FileSystemRepository implements ContentRepository {
|
|||
|
||||
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 class FileSystemRepository implements ContentRepository {
|
|||
}
|
||||
|
||||
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 class FileSystemRepository implements ContentRepository {
|
|||
}
|
||||
|
||||
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 class FileSystemRepository implements ContentRepository {
|
|||
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 class FileSystemRepository implements ContentRepository {
|
|||
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 class FileSystemRepository implements ContentRepository {
|
|||
return out;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void purge() {
|
||||
// delete all content from repositories
|
||||
|
@ -1035,7 +1045,7 @@ public class FileSystemRepository implements ContentRepository {
|
|||
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 @@ public class FileSystemRepository implements ContentRepository {
|
|||
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 @@ public class FileSystemRepository implements ContentRepository {
|
|||
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 class FileSystemRepository implements ContentRepository {
|
|||
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 class FileSystemRepository implements ContentRepository {
|
|||
|
||||
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 class FileSystemRepository implements ContentRepository {
|
|||
LOG.warn("", e);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (remove(claim)) {
|
||||
successCount++;
|
||||
}
|
||||
} else if (remove(claim)) {
|
||||
successCount++;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1571,7 +1579,9 @@ public class FileSystemRepository implements ContentRepository {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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 class FileSystemRepository implements ContentRepository {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private static class ClaimLengthPair {
|
||||
|
||||
private final ResourceClaim claim;
|
||||
private final Long length;
|
||||
|
||||
|
|
|
@ -56,24 +56,33 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* 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 <code>nifi.volatile.content.repository.max.size</code> 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
|
||||
* <code>nifi.volatile.content.repository.max.size</code> property in the NiFi
|
||||
* properties with a default of 100 MB), one of two situations will occur:
|
||||
* </p>
|
||||
*
|
||||
* <ul>
|
||||
* <li><b>Backup Repository:</b> 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.
|
||||
* <li><b>Backup Repository:</b> 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.
|
||||
* </li>
|
||||
* <li>
|
||||
* <b>Without Backup Repository:</b> If no Backup Repository has been specified, when the threshold is exceeded, an IOException will be thrown.
|
||||
* <b>Without Backup Repository:</b> If no Backup Repository has been specified,
|
||||
* when the threshold is exceeded, an IOException will be thrown.
|
||||
* </li>
|
||||
* </ul>
|
||||
*
|
||||
* <p>
|
||||
* When a Content Claim is created via the {@link #create(boolean)} method, if the <code>lossTolerant</code> flag is set to <code>false</code>, 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 <code>lossTolerant</code> flag is set to <code>false</code>, 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.
|
||||
* </p>
|
||||
*/
|
||||
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 class VolatileContentRepository implements ContentRepository {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class VolatileContentRepository implements ContentRepository {
|
|||
|
||||
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);
|
||||
|
|
|
@ -63,12 +63,21 @@ import org.wali.WriteAheadRepository;
|
|||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* We expose a property named <code>nifi.flowfile.repository.always.sync</code> that is a boolean value indicating whether or not to force WALI to sync with disk on each update. By default, the value
|
||||
* is <code>false</code>. 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 <code>nifi.flowfile.repository.always.sync</code>
|
||||
* that is a boolean value indicating whether or not to force WALI to sync with
|
||||
* disk on each update. By default, the value is <code>false</code>. 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.
|
||||
* </p>
|
||||
*/
|
||||
public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncListener {
|
||||
|
@ -112,15 +121,24 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
|
|||
// on restart.
|
||||
private final ConcurrentMap<Integer, BlockingQueue<ResourceClaim>> 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 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void onSync(final int partitionIndex) {
|
||||
final BlockingQueue<ResourceClaim> claimQueue = claimsAwaitingDestruction.get(Integer.valueOf(partitionIndex));
|
||||
|
@ -282,7 +299,9 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
|
|||
}
|
||||
|
||||
private static class WriteAheadRecordSerde implements SerDe<RepositoryRecord> {
|
||||
|
||||
private static final int CURRENT_ENCODING_VERSION = 9;
|
||||
|
||||
public static final byte ACTION_CREATE = 0;
|
||||
|
@ -551,7 +571,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
|
|||
|
||||
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
|
||||
|
@ -662,7 +682,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
|
|||
|
||||
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
|
||||
|
|
|
@ -62,7 +62,8 @@ import org.slf4j.Logger;
|
|||
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 final class StandardProcessScheduler implements ProcessScheduler {
|
|||
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 final class StandardProcessScheduler implements ProcessScheduler {
|
|||
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 final class StandardProcessScheduler implements ProcessScheduler {
|
|||
* 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 final class StandardProcessScheduler implements ProcessScheduler {
|
|||
* 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 final class StandardProcessScheduler implements ProcessScheduler {
|
|||
* 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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.ControllerService;
|
|||
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 class StandardControllerServiceInitializationContext implements Controlle
|
|||
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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.nifi.registry.VariableRegistry;
|
|||
|
||||
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 class StandardControllerServiceProvider implements ControllerServiceProvi
|
|||
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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -26,14 +26,16 @@ import org.jasypt.exceptions.EncryptionOperationNotPossibleException;
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Instance of this class are thread-safe</p>
|
||||
*
|
||||
* <p>
|
||||
* The encryption provider and algorithm is configured using the application properties:
|
||||
* The encryption provider and algorithm is configured using the application
|
||||
* properties:
|
||||
* <ul>
|
||||
* <li>nifi.sensitive.props.provider</li>
|
||||
* <li>nifi.sensitive.props.algorithm</li>
|
||||
|
@ -71,18 +73,21 @@ public final class StringEncryptor {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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");
|
||||
|
|
|
@ -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 final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
|
|||
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 final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
|
|||
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)) {
|
||||
|
|
|
@ -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 class StandardProcessorInitializationContext implements ProcessorInitiali
|
|||
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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -76,8 +76,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
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<String> name = new AtomicReference<>();
|
||||
private final AtomicReference<Position> 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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 <code>true</code> if an Output Port exists with the given ID, <code>false</code> otherwise.
|
||||
* @return <code>true</code> if an Output Port exists with the given ID,
|
||||
* <code>false</code> otherwise.
|
||||
*/
|
||||
public boolean containsOutputPort(final String id) {
|
||||
readLock.lock();
|
||||
|
@ -465,8 +472,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
*
|
||||
*
|
||||
* @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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
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 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
|
||||
/**
|
||||
* @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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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<RemoteGroupPort> getInputPorts() {
|
||||
|
@ -659,11 +674,13 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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<RemoteGroupPort> getOutputPorts() {
|
||||
|
@ -772,10 +790,9 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
|
||||
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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
// 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 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
|
||||
@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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
*/
|
||||
*/
|
||||
authorizationIssue = e.getDescription();
|
||||
|
||||
} else if (e.getResponseCode() == FORBIDDEN_STATUS_CODE) {
|
||||
|
@ -1192,7 +1208,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
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 class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
private File getPeerPersistenceFile() {
|
||||
final File stateDir = NiFiProperties.getInstance().getPersistentStateDirectory();
|
||||
final File stateDir = nifiProperties.getPersistentStateDirectory();
|
||||
return new File(stateDir, getIdentifier() + ".peers");
|
||||
}
|
||||
|
||||
|
|
|
@ -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<String, Connection> connectionMocksById = [:]
|
||||
def Map<String, List<Position>> 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
|
||||
|
|
|
@ -74,7 +74,7 @@ public class StandardFlowServiceTest {
|
|||
|
||||
@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);
|
||||
|
|
|
@ -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 class TestFileSystemSwapManager {
|
|||
|
||||
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 class TestFileSystemSwapManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public class NopResourceClaimManager implements ResourceClaimManager {
|
||||
|
||||
@Override
|
||||
|
@ -162,8 +160,8 @@ public class TestFileSystemSwapManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private static class TestFlowFile implements FlowFileRecord {
|
||||
|
||||
private static final AtomicLong idGenerator = new AtomicLong(0L);
|
||||
|
||||
private final long id = idGenerator.getAndIncrement();
|
||||
|
@ -172,13 +170,11 @@ public class TestFileSystemSwapManager {
|
|||
private final Map<String, String> attributes;
|
||||
private final long size;
|
||||
|
||||
|
||||
public TestFlowFile(final Map<String, String> attributes, final long size) {
|
||||
this.attributes = attributes;
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public long getId() {
|
||||
return id;
|
||||
|
|
|
@ -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.File;
|
|||
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<String, String> 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 class TestFlowController {
|
|||
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 class TestFlowController {
|
|||
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());
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.nifi.processor.ProcessSession;
|
|||
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);
|
||||
|
|
|
@ -55,6 +55,8 @@ import ch.qos.logback.classic.Level;
|
|||
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 class TestFileSystemRepository {
|
|||
testAppender.setName("Test");
|
||||
testAppender.start();
|
||||
root.addAppender(testAppender);
|
||||
final Map<String, String> 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 class TestFileSystemRepository {
|
|||
repository.shutdown();
|
||||
Thread.sleep(1000L);
|
||||
|
||||
repository = new FileSystemRepository();
|
||||
repository = new FileSystemRepository(nifiProperties);
|
||||
repository.initialize(new StandardResourceClaimManager());
|
||||
repository.purge();
|
||||
|
||||
|
@ -206,7 +200,6 @@ public class TestFileSystemRepository {
|
|||
assertNotSame(claim1.getResourceClaim(), claim2.getResourceClaim());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testWriteWithNoContent() throws IOException {
|
||||
final ContentClaim claim1 = repository.create(false);
|
||||
|
@ -314,7 +307,6 @@ public class TestFileSystemRepository {
|
|||
assertTrue(Arrays.equals(expected, baos.toByteArray()));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testImportFromStream() throws IOException {
|
||||
final ContentClaim claim = repository.create(false);
|
||||
|
@ -452,7 +444,7 @@ public class TestFileSystemRepository {
|
|||
// 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 @@ public class TestFileSystemRepository {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testWriteCannotProvideNullOutput() throws IOException {
|
||||
FileSystemRepository repository = null;
|
||||
|
@ -504,7 +495,7 @@ public class TestFileSystemRepository {
|
|||
// 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 @@ public class TestFileSystemRepository {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class TestFileSystemRepository {
|
|||
// 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) {
|
||||
|
|
|
@ -80,6 +80,7 @@ import org.apache.nifi.provenance.ProvenanceEventRecord;
|
|||
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 @@ public class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
assertEquals(0, numClaims);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testModifyContentThenRollback() throws IOException {
|
||||
assertEquals(0, contentRepo.getExistingClaims().size());
|
||||
|
@ -283,10 +283,10 @@ public class TestStandardProcessSession {
|
|||
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 @@ public class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
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<InputStream> 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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
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<List<FlowFileRecord>>() {
|
||||
int iterations = 0;
|
||||
|
@ -930,7 +929,7 @@ public class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
}
|
||||
|
||||
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 class TestStandardProcessSession {
|
|||
@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 class TestStandardProcessSession {
|
|||
}
|
||||
|
||||
private static class MockFlowFileRepository implements FlowFileRepository {
|
||||
|
||||
private boolean failOnUpdate = false;
|
||||
private final AtomicLong idGenerator = new AtomicLong(0L);
|
||||
|
||||
|
|
|
@ -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.IOException;
|
|||
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 class TestVolatileContentRepository {
|
|||
|
||||
@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<String, String> 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 class TestVolatileContentRepository {
|
|||
|
||||
@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<String, String> 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 class TestVolatileContentRepository {
|
|||
|
||||
@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<String, String> 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);
|
||||
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.nifi.controller.repository.claim.StandardContentClaim;
|
|||
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 class TestWriteAheadFlowFileRepository {
|
|||
// 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 class TestWriteAheadFlowFileRepository {
|
|||
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 class TestWriteAheadFlowFileRepository {
|
|||
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 class TestWriteAheadFlowFileRepository {
|
|||
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 class TestWriteAheadFlowFileRepository {
|
|||
}
|
||||
|
||||
private static class TestQueueProvider implements QueueProvider {
|
||||
|
||||
private List<Connection> connectionList = new ArrayList<>();
|
||||
|
||||
public void addConnection(final Connection connection) {
|
||||
|
@ -249,6 +250,7 @@ public class TestWriteAheadFlowFileRepository {
|
|||
}
|
||||
|
||||
private static class MockFlowFileSwapManager implements FlowFileSwapManager {
|
||||
|
||||
private final Map<FlowFileQueue, Map<String, List<FlowFileRecord>>> swappedRecords = new HashMap<>();
|
||||
|
||||
@Override
|
||||
|
|
|
@ -56,8 +56,10 @@ import java.util.ArrayList;
|
|||
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 class TestProcessorLifecycle {
|
|||
|
||||
@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 class TestProcessorLifecycle {
|
|||
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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
*/
|
||||
@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 class TestProcessorLifecycle {
|
|||
|
||||
/**
|
||||
* 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 @@ public class TestProcessorLifecycle {
|
|||
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<String, String> 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 @@ public class TestProcessorLifecycle {
|
|||
/**
|
||||
*/
|
||||
public static class TestProcessor extends AbstractProcessor {
|
||||
|
||||
private Runnable onScheduleCallback;
|
||||
private Runnable onUnscheduleCallback;
|
||||
private Runnable onStopCallback;
|
||||
|
@ -750,8 +770,8 @@ public class TestProcessorLifecycle {
|
|||
.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 class TestProcessorLifecycle {
|
|||
/**
|
||||
*/
|
||||
private static class EmptyRunnable implements Runnable {
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
|
||||
|
@ -787,6 +808,7 @@ public class TestProcessorLifecycle {
|
|||
/**
|
||||
*/
|
||||
private static class BlockingInterruptableRunnable implements Runnable {
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
|
@ -800,6 +822,7 @@ public class TestProcessorLifecycle {
|
|||
/**
|
||||
*/
|
||||
private static class BlockingUninterruptableRunnable implements Runnable {
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (true) {
|
||||
|
@ -815,6 +838,7 @@ public class TestProcessorLifecycle {
|
|||
/**
|
||||
*/
|
||||
private static class RandomOrFixedDelayedRunnable implements Runnable {
|
||||
|
||||
private final int delayLimit;
|
||||
private final boolean randomDelay;
|
||||
|
||||
|
@ -823,6 +847,7 @@ public class TestProcessorLifecycle {
|
|||
this.randomDelay = randomDelay;
|
||||
}
|
||||
Random random = new Random();
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
|
|
|
@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals;
|
|||
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.junit.Test;
|
|||
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 class TestStandardProcessScheduler {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class TestStandardProcessScheduler {
|
|||
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 class TestStandardProcessScheduler {
|
|||
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 class TestStandardProcessScheduler {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private static class ServiceReferencingProcessor extends AbstractProcessor {
|
||||
|
||||
static final PropertyDescriptor SERVICE_DESC = new PropertyDescriptor.Builder()
|
||||
.name("service")
|
||||
.identifiesControllerService(NoStartService.class)
|
||||
|
@ -195,15 +198,6 @@ public class TestStandardProcessScheduler {
|
|||
}
|
||||
}
|
||||
|
||||
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 @@ public class TestStandardProcessScheduler {
|
|||
@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 class TestStandardProcessScheduler {
|
|||
@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 class TestStandardProcessScheduler {
|
|||
@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 class TestStandardProcessScheduler {
|
|||
@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 class TestStandardProcessScheduler {
|
|||
@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 class TestStandardProcessScheduler {
|
|||
@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 class TestStandardProcessScheduler {
|
|||
|
||||
/**
|
||||
* 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 class TestStandardProcessScheduler {
|
|||
}
|
||||
|
||||
public static class FailingService extends AbstractControllerService {
|
||||
|
||||
@OnEnabled
|
||||
public void enable(final ConfigurationContext context) {
|
||||
throw new RuntimeException("intentional");
|
||||
|
@ -453,6 +449,7 @@ public class TestStandardProcessScheduler {
|
|||
}
|
||||
|
||||
public static class RandomShortDelayEnablingService extends AbstractControllerService {
|
||||
|
||||
private final Random random = new Random();
|
||||
|
||||
@OnEnabled
|
||||
|
@ -490,6 +487,7 @@ public class TestStandardProcessScheduler {
|
|||
}
|
||||
|
||||
public static class LongEnablingService extends AbstractControllerService {
|
||||
|
||||
private final AtomicInteger enableCounter = new AtomicInteger();
|
||||
private final AtomicInteger disableCounter = new AtomicInteger();
|
||||
|
||||
|
@ -520,6 +518,6 @@ public class TestStandardProcessScheduler {
|
|||
}
|
||||
|
||||
private ProcessScheduler createScheduler() {
|
||||
return new StandardProcessScheduler(null, null, stateMgrProvider, variableRegistry);
|
||||
return new StandardProcessScheduler(null, null, stateMgrProvider, variableRegistry, nifiProperties);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.nifi.controller.service;
|
|||
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 class StandardControllerServiceProviderTest {
|
|||
@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 class StandardControllerServiceProviderTest {
|
|||
public void testCallImplementationInitialized() throws InitializationException {
|
||||
implementation.initialize(null);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,12 +47,14 @@ import org.apache.nifi.groups.ProcessGroup;
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
|
||||
@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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@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 class TestStandardControllerServiceProvider {
|
|||
|
||||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
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 class TestStandardControllerServiceProvider {
|
|||
|
||||
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());
|
||||
|
|
|
@ -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 @@ import org.testng.Assert;
|
|||
|
||||
public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
||||
|
||||
private StateProvider provider;
|
||||
private TestingServer zkServer;
|
||||
private volatile StateProvider provider;
|
||||
private volatile TestingServer zkServer;
|
||||
|
||||
private static final Map<PropertyDescriptor, String> 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 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
|||
@After
|
||||
public void clear() throws IOException {
|
||||
try {
|
||||
getProvider().onComponentRemoved(componentId);
|
||||
getProvider().disable();
|
||||
getProvider().shutdown();
|
||||
if (provider != null) {
|
||||
provider.onComponentRemoved(componentId);
|
||||
provider.disable();
|
||||
provider.shutdown();
|
||||
}
|
||||
} finally {
|
||||
if (zkServer != null) {
|
||||
zkServer.stop();
|
||||
|
@ -110,13 +110,11 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected StateProvider getProvider() {
|
||||
return provider;
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout = 20000)
|
||||
public void testStateTooLargeExceptionThrownOnSetState() throws InterruptedException {
|
||||
final Map<String, String> state = new HashMap<>();
|
||||
|
@ -152,7 +150,6 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout = 20000)
|
||||
public void testStateTooLargeExceptionThrownOnReplace() throws IOException, InterruptedException {
|
||||
final Map<String, String> state = new HashMap<>();
|
||||
|
|
|
@ -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 java.util.Collections;
|
|||
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 class TestContinuallyRunProcessorTask {
|
|||
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));
|
||||
|
|
|
@ -34,11 +34,14 @@ import org.apache.nifi.reporting.ReportingTask;
|
|||
|
||||
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 @@ public class NarThreadContextClassLoader extends URLClassLoader {
|
|||
}
|
||||
}
|
||||
|
||||
public static <T> T createInstance(final String implementationClassName, final Class<T> 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 <T> 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> T createInstance(final String implementationClassName, final Class<T> 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 class NarThreadContextClassLoader extends URLClassLoader {
|
|||
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -21,11 +21,8 @@ import org.junit.Assert;
|
|||
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.HashSet;
|
|||
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 class NarUnpackerTest {
|
|||
@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 class NarUnpackerTest {
|
|||
@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<String, String> 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 class NarUnpackerTest {
|
|||
nonExistantDir.delete();
|
||||
nonExistantDir.deleteOnExit();
|
||||
|
||||
NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties");
|
||||
properties.setProperty("nifi.nar.library.directory.alt", nonExistantDir.toString());
|
||||
final Map<String, String> 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 class NarUnpackerTest {
|
|||
nonDir.createNewFile();
|
||||
nonDir.deleteOnExit();
|
||||
|
||||
NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties");
|
||||
properties.setProperty("nifi.nar.library.directory.alt", nonDir.toString());
|
||||
final Map<String, String> 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<String, String> others) {
|
||||
String filePath;
|
||||
try {
|
||||
filePath = NarUnpackerTest.class.getResource(propertiesFile).toURI().getPath();
|
||||
|
@ -181,34 +183,6 @@ public class NarUnpackerTest {
|
|||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -224,7 +224,7 @@ public class NiFi {
|
|||
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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
}
|
||||
});
|
||||
|
||||
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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
}
|
||||
|
||||
private class TransactionWrapper {
|
||||
|
||||
private final FlowFileTransaction transaction;
|
||||
private final HandshakeProperties handshakenProperties;
|
||||
private long lastCommunicationAt;
|
||||
|
@ -129,7 +129,7 @@ public class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
try {
|
||||
Set<String> 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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
} 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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
if(transactionMaintenanceTask != null) {
|
||||
if (transactionMaintenanceTask != null) {
|
||||
logger.debug("Stopping transactionMaintenanceTask...");
|
||||
transactionMaintenanceTask.cancel(true);
|
||||
}
|
||||
|
@ -191,10 +190,9 @@ public class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
|
||||
/**
|
||||
* @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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
}
|
||||
|
||||
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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
}
|
||||
|
||||
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 class HttpRemoteSiteListener implements RemoteSiteListener {
|
|||
}
|
||||
|
||||
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);
|
||||
|
|
|
@ -122,8 +122,8 @@ public class RemoteResourceFactory extends RemoteResourceInitiator {
|
|||
}
|
||||
|
||||
public static <T extends VersionedRemoteResource> T
|
||||
receiveResourceNegotiation(final Class<T> cls, final DataInputStream dis, final DataOutputStream dos, final Class<?>[] constructorArgClasses, final Object[] constructorArgs)
|
||||
throws IOException, HandshakeException {
|
||||
receiveResourceNegotiation(final Class<T> 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 {
|
||||
|
|
|
@ -46,6 +46,8 @@ import java.util.Arrays;
|
|||
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<ProcessGroup> 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 class SocketRemoteSiteListener implements RemoteSiteListener {
|
|||
protocol.getPort().receiveFlowFiles(peer, protocol);
|
||||
break;
|
||||
case REQUEST_PEER_LIST:
|
||||
protocol.sendPeerList(peer, nodeInformant == null ? Optional.empty() : Optional.of(nodeInformant.getNodeInformation()));
|
||||
final Optional<ClusterNodeInformation> 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);
|
||||
|
|
|
@ -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<SiteToSiteClient> 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 class StandardRemoteGroupPort extends RemoteGroupPort {
|
|||
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 class StandardRemoteGroupPort extends RemoteGroupPort {
|
|||
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 class StandardRemoteGroupPort extends RemoteGroupPort {
|
|||
|
||||
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 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
|
|||
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 class StandardRemoteGroupPort extends RemoteGroupPort {
|
|||
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) {
|
||||
|
|
|
@ -201,7 +201,7 @@ public abstract class AbstractFlowFileServerProtocol implements ServerProtocol {
|
|||
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 @@ public abstract class AbstractFlowFileServerProtocol implements ServerProtocol {
|
|||
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 abstract class AbstractFlowFileServerProtocol implements ServerProtocol {
|
|||
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 @@ public abstract class AbstractFlowFileServerProtocol implements ServerProtocol {
|
|||
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();
|
||||
}
|
||||
|
|
|
@ -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<FlowFile> flowFilesSent, String calculatedCRC) {
|
||||
|
|
|
@ -28,7 +28,6 @@ public class HandshakeProperties {
|
|||
private long batchBytes = 0L;
|
||||
private long batchDurationNanos = 0L;
|
||||
|
||||
|
||||
public String getCommsIdentifier() {
|
||||
return commsIdentifier;
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import java.io.DataInputStream;
|
|||
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 @@ public class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerPr
|
|||
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 @@ public class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerPr
|
|||
|
||||
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 @@ public class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerPr
|
|||
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 @@ public class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerPr
|
|||
// 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 class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerPr
|
|||
}
|
||||
|
||||
@Override
|
||||
public void sendPeerList(final Peer peer, final Optional<ClusterNodeInformation> clusterNodeInformation) throws IOException {
|
||||
public void sendPeerList(Peer peer, Optional<ClusterNodeInformation> clusterNodeInfo, String remoteInputHost,
|
||||
int remoteInputPort, int remoteInputHttpPort, boolean isSiteToSiteSecure) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.nifi.remote.protocol.CommunicationsSession;
|
|||
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 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
|||
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 class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public RequestType getRequestType(final Peer peer) throws IOException {
|
||||
if (!handshakeCompleted) {
|
||||
|
@ -154,7 +152,13 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
|||
}
|
||||
|
||||
@Override
|
||||
public void sendPeerList(final Peer peer, final Optional<ClusterNodeInformation> clusterNodeInfo) throws IOException {
|
||||
public void sendPeerList(
|
||||
final Peer peer,
|
||||
final Optional<ClusterNodeInformation> 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 class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
|||
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 class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
|||
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 class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
|||
return RESOURCE_NAME;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public VersionNegotiator getVersionNegotiator() {
|
||||
return versionNegotiator;
|
||||
|
|
|
@ -39,7 +39,7 @@ public class TestHttpRemoteSiteListener {
|
|||
|
||||
@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 class TestHttpRemoteSiteListener {
|
|||
|
||||
@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 class TestHttpRemoteSiteListener {
|
|||
|
||||
@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 class TestHttpRemoteSiteListener {
|
|||
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 class TestHttpRemoteSiteListener {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -43,6 +43,7 @@ import java.net.URI;
|
|||
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 @@ public class TestStandardRemoteGroupPort {
|
|||
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();
|
||||
|
|
|
@ -76,7 +76,7 @@ package org.apache.nifi.remote.io.socket;
|
|||
// 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);
|
||||
|
|
|
@ -67,7 +67,7 @@ package org.apache.nifi.remote.io.socket.ssl;
|
|||
// 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 @@ package org.apache.nifi.remote.io.socket.ssl;
|
|||
// 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 @@ package org.apache.nifi.remote.io.socket.ssl;
|
|||
// 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 @@ package org.apache.nifi.remote.io.socket.ssl;
|
|||
// 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 @@ package org.apache.nifi.remote.io.socket.ssl;
|
|||
// 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 @@ package org.apache.nifi.remote.io.socket.ssl;
|
|||
// 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);
|
||||
|
|
|
@ -90,7 +90,7 @@ public class TestHttpFlowFileServerProtocol {
|
|||
|
||||
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 class TestHttpFlowFileServerProtocol {
|
|||
}
|
||||
|
||||
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 @@ public class TestHttpFlowFileServerProtocol {
|
|||
|
||||
@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 class TestHttpFlowFileServerProtocol {
|
|||
}
|
||||
|
||||
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 @@ public class TestHttpFlowFileServerProtocol {
|
|||
|
||||
@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/"
|
||||
|
|
|
@ -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 @@ import static org.mockito.Mockito.verify;
|
|||
|
||||
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<String, String> 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 class JettyServerTest {
|
|||
// Expect that with no KeyStore password, we will only need to set Key Manager Password
|
||||
String testKeyPassword = "testKeyPassword";
|
||||
|
||||
NiFiProperties nifiProperties = createNifiProperties();
|
||||
final Map<String, String> 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 class JettyServerTest {
|
|||
// Expect that with no KeyPassword, we use the same one from the KeyStore
|
||||
String testKeystorePassword = "testKeystorePassword";
|
||||
|
||||
NiFiProperties nifiProperties = createNifiProperties();
|
||||
final Map<String, String> 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<NiFiProperties> constructor = NiFiProperties.class.getDeclaredConstructor();
|
||||
constructor.setAccessible(true);
|
||||
return constructor.newInstance();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -92,6 +92,7 @@ import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERT
|
|||
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 @@ public class DataTransferResource extends ApplicationResource {
|
|||
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 class DataTransferResource extends ApplicationResource {
|
|||
@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 class DataTransferResource extends ApplicationResource {
|
|||
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 class DataTransferResource extends ApplicationResource {
|
|||
}
|
||||
|
||||
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 @@ public class DataTransferResource extends ApplicationResource {
|
|||
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 class DataTransferResource extends ApplicationResource {
|
|||
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 class DataTransferResource extends ApplicationResource {
|
|||
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 @@ public class DataTransferResource extends ApplicationResource {
|
|||
return Response.ok(entity).build();
|
||||
}
|
||||
|
||||
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_OCTET_STREAM)
|
||||
|
@ -601,19 +600,18 @@ public class DataTransferResource extends ApplicationResource {
|
|||
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 class DataTransferResource extends ApplicationResource {
|
|||
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 class DataTransferResource extends ApplicationResource {
|
|||
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 class DataTransferResource extends ApplicationResource {
|
|||
}
|
||||
|
||||
private class ValidateRequestResult {
|
||||
|
||||
private Integer transportProtocolVersion;
|
||||
private Response errResponse;
|
||||
}
|
||||
|
@ -824,9 +822,7 @@ public class DataTransferResource extends ApplicationResource {
|
|||
return result;
|
||||
}
|
||||
|
||||
|
||||
// setters
|
||||
|
||||
public void setAuthorizer(Authorizer authorizer) {
|
||||
this.authorizer = authorizer;
|
||||
}
|
||||
|
|
|
@ -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.List;
|
|||
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 @@ public class SiteToSiteResource extends ApplicationResource {
|
|||
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 class SiteToSiteResource extends ApplicationResource {
|
|||
return clusterContext(noCache(Response.ok(entity))).build();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns the available Peers and its status of this NiFi.
|
||||
*
|
||||
|
@ -188,15 +178,15 @@ public class SiteToSiteResource extends ApplicationResource {
|
|||
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 class SiteToSiteResource extends ApplicationResource {
|
|||
}
|
||||
|
||||
// setters
|
||||
|
||||
public void setServiceFacade(final NiFiServiceFacade serviceFacade) {
|
||||
this.serviceFacade = serviceFacade;
|
||||
}
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.nifi.nar.NarClassLoaders;
|
|||
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 class AccessControlHelper {
|
|||
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<String, String> 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());
|
||||
|
|
|
@ -64,8 +64,9 @@ public class ITAccessTokenEndpoint {
|
|||
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<String, String> 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());
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.nifi.integration.util;
|
|||
|
||||
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 class NiFiTestServer {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 <T> 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> T getSpringBean(String beanName, Class<T> clazz) {
|
||||
ServletContext servletContext = webappContext.getServletHandler().getServletContext();
|
||||
|
|
|
@ -178,7 +178,7 @@ public class TestDataTransferResource {
|
|||
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 class TestDataTransferResource {
|
|||
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 class TestDataTransferResource {
|
|||
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 class TestDataTransferResource {
|
|||
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 class TestDataTransferResource {
|
|||
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 class TestDataTransferResource {
|
|||
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 class TestDataTransferResource {
|
|||
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 class TestDataTransferResource {
|
|||
|
||||
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 @@ public class TestDataTransferResource {
|
|||
return serverProtocol;
|
||||
}
|
||||
};
|
||||
resource.setProperties(NiFiProperties.getInstance());
|
||||
resource.setProperties(NiFiProperties.createBasicNiFiProperties(null, null));
|
||||
return resource;
|
||||
}
|
||||
}
|
|
@ -151,12 +151,12 @@ public class TestSiteToSiteResource {
|
|||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
|
|
@ -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<String> getPropertyKeys() {
|
||||
return ["A", "set"]
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue