You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2020/03/17 12:47:36 UTC

[nifi] branch support/nifi-1.11.x updated (b5ff8f3 -> 4e4fa38)

This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a change to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git.


    from b5ff8f3  NIFI-7181 update docker versions for next rel
     new 1fed416  NIFI-7178 - Handle the case when schema is not available
     new d17a5aa  NIFI-7185 relaxed timing constrained for builds on lower resources environments like our Github CI builds
     new fa0632b  NIFI-7183 - This closes #4073. Improve ReplaceText when removing FF's content
     new 98e4246  NIFI-7164 Upgrade shyiko/mysql-binlog-connector-java dependency
     new 211b0e8  NIFI-7201 - Update build to latest apache-maven-parent and split Github Actions builds to include OSX and distribute localization
     new 024d727  NIFI-7205 NIFI-7206
     new 1d10b51  NIFI-6856 - Make client ID a non-required field for the MQTTConsume and MQTTProduce processors. Generates a random ID if not set.
     new 91f3eb0  NIFI-5644 Fixed typo in getWrappedQuery method of AbstractDatabaseFetchProcessor class
     new 089ca02  NIFI-7218 Fixed typo in Overview docs. (#4107)
     new 3106604  NIFI-7121 Updated comment to state a 'static' salt is used in the constructor. (#4098)
     new 1f41c21  NIFI-4970 - EOF Exception in InvokeHttp when body's response is empty with gzip
     new e1c2656  NIFI-7208: Fixed PutSQL/JdbcCommon handling of timestamps (nanoseconds, e.g.)
     new 65ffd45  NIFI-7227 Fixed typo in Global Access Policy table (#4112)
     new 4c78d1e  NIFI-7222 Cleaned up API for FTP/SFTP remote file retrieval and ensure we close remote file resources for SFTP pulls in particular
     new ad892a4  NIFI-7226: Add Connection Factory configuration properties to PublishJMS and ConsumeJMS processors
     new 5c3f983  NIFI-7232 if users do not supply a remote path PutSFTP with conflict resolution will fail with an NPE
     new 38124f2  Fixed unit test failed with NIFI-7232
     new 20c086c  NIFI-7191 Conditionally disable docker integration tests
     new ca27a6c  NIFI-7231: move controller service validation out of synchronized block for enabling
     new c82c1db  NIFI-7197 - In-place replacement in LookupRecord processor
     new b7cbd23  NIFI-7224 Protecting against possible NPE in ImportFlowVersion command in CLI
     new f1d1c0f  NIFI-7195 - Catch MongoException to route flow files to failure
     new ae8e752  Fiz writeBatch to avoid cycle in provenance graph
     new a655be9  NIFI-7210 - added PG path in bulletins for S2S Bulletin RT
     new 6f5bd17  NIFI-7200: Revert "NIFI-6530 - HTTP SiteToSite server returns 201 in case no data is available"
     new aa2f561  NIFI-4970 - Add a property to deal with empty GZIP HTTP response
     new 716e32e  NIFI-7239: Upgrade the Hive 3 bundle to use Apache Hive 3.1.2
     new 4412460  NIFI-7050 ConsumeJMS is not yielded in case of exception
     new 833eea4  NIFI-7245: JMS processors yield when connection factory initialisation failed
     new 450bcd0  unit test reproducing the issue
     new 7839493  Fixed bug in JsonRowRecordReader when reading a 'raw' record with a schema that indicates that a field should be a Map. Also updated unit test to explicitly define schema, since schema inference will never return a Map but rather a Record
     new 6a3f6f6  NIFI-7242: When a Parameter is changed, any property referencing that parameter should have its #onPropertyModified method called. Also renamed Accumulo tests to integration tests because they start embedded servers and connect to them, which caused failures in my environment. Also fixed a bug in TestLengthDelimitedJournal because it was resulting in failures when building locally as well.
     new 950873c  NIFI-7119 Implement boundary checking for Argon2 cost parameters (#4111)
     new eeb7d69  NIFI-7241: When updating Process Group to match VersionedProcessGroup, remove any connections before recursing into child groups. This ensures that if a Port exists in child group A and is connected to a port in child group B, if the VersionedProcessGroup indicates to remove the port, that connection will be removed before attempting to remove the port. Updating and adding connections must still be done last, after all components have been added. But missing connections  [...]
     new c44204d  NIFI-7248: Atlas reporting task handles PutHive3Streaming
     new 0d5cd83  NIFI-7244 Updated all tests which dont run well on windows to either work or be ignored on windows Also dealt with unreliable tests which depend on timing by ignoring them or converting to IT.
     new 85efc09  NIFI-6742 Use JUnit TemporaryFolder when creating test databases
     new 42b453c  NIFI-7251: Upgrade hadoop-client version to 3.2.1 to avoid the regression bug
     new 1f2b68c  NIFI-7229 - Upgrade jackson-databind direct dependencies
     new 76c5335  NIFI-7249: Force String keys in maps in DataTypeUtils.inferDataType()
     new 24ece00  NIFI-7250 activate user.timezone appropriate to each region NIFI-7250 fix a test which appears brittle at least on windows builds on slow environments NIFI-7250 activated a timezone run for AU Australia/Melbourne which exposed a poor magic number and needless assertion but interesting results worth keeping
     new c3457da  NIFI-7256: This closes #4142. Fixed thresholds in unit test. Instead of assuming that multiple runs of the processor will occur within 100 milliseconds, allowed the multiple runs to occur within 3 mins of one another.
     new d6655bd  NIFI-7208: Restore default timezone in JdbcCommon
     new 67db741  NIFI-7223 - Fixed a minor issue where the OkHttpReplicationClient class loaded blank properties as empty string instead of an expected null value. Added a isNotBlank check. Added unit tests for replication client and HTTPNotificationService. NIFI-7223 - Renamed some variables and methods. NIFI-7223 - Removed unused dependency. Corrected security properties in administration-guide.
     new 554a637b NIFI-7223 [WIP] Resolved compilation issues in unit test on OpenJDK 11 by removing Sun security class references. Added OkHttpReplicationClient#isTLSConfigured() method. Added unit test. NIFI-7223 Fixed remaining unit tests for TLS regression. Renamed tests for clarity.
     new bdef4b4  NIFI-7223 - Added another test for when keyPasswd is not present.
     new 4e4fa38  NIFI-7223 Resolved merge conflicts from additional test case for null key password.

The 47 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .github/workflows/ci-workflow.yml                  | 157 +++++++++--
 .../java/org/apache/nifi/reporting/Bulletin.java   |   9 +
 nifi-bootstrap/pom.xml                             |   5 +
 .../http/TestHttpNotificationServiceSSL.java       | 190 ++++++++++++-
 .../attribute/expression/language/TestQuery.java   |   5 +
 .../language/TestStandardPreparedQuery.java        |   5 +
 .../org/apache/nifi/parameter/ParameterLookup.java |  14 +
 .../serialization/record/util/DataTypeUtils.java   |  18 +-
 .../serialization/record/TestDataTypeUtils.java    |  31 +++
 nifi-commons/nifi-rocksdb-utils/pom.xml            |   6 +
 .../apache/nifi/rocksdb/TestRocksDBMetronome.java  |  14 +-
 nifi-commons/nifi-security-utils/pom.xml           |   4 -
 .../security/util/crypto/Argon2SecureHasher.java   | 140 +++++++++-
 .../util/crypto/Argon2SecureHasherTest.groovy      | 156 +++++++++++
 .../apache/nifi/remote/client/PeerSelector.java    |  22 +-
 .../apache/nifi/remote/client/http/HttpClient.java |  22 +-
 .../http/TransportProtocolVersionNegotiator.java   |   1 -
 .../client/socket/EndpointConnectionPool.java      |  19 +-
 .../nifi/remote/client/socket/SocketClient.java    |   6 +-
 .../nifi/remote/exception/NoContentException.java  |  39 ---
 .../remote/exception/NoValidPeerException.java     |  40 ---
 .../protocol/socket/SocketClientTransaction.java   |   4 -
 .../nifi/remote/util/SiteToSiteRestApiClient.java  |   6 +-
 .../nifi/remote/client/TestPeerSelector.java       |  31 +--
 .../nifi/remote/client/http/TestHttpClient.java    |  63 +----
 .../socket/TestSocketClientTransaction.java        |  17 +-
 .../nifi/wali/TestLengthDelimitedJournal.java      |  13 +-
 nifi-docker/dockerhub/pom.xml                      |  45 +++
 nifi-docker/dockermaven/pom.xml                    |  45 +++
 .../src/main/asciidoc/administration-guide.adoc    |   7 +-
 nifi-docs/src/main/asciidoc/overview.adoc          |   2 +-
 .../java/org/apache/nifi/spark/NiFiReceiver.java   |   7 -
 .../org/apache/nifi/reporting/BulletinFactory.java |  15 +
 .../{TestPutRecord.java => PutRecordIT.java}       |   5 +-
 .../{TestScanAccumulo.java => ScanAccumuloIT.java} |   5 +-
 .../provenance/analyzer/PutHiveStreaming.java      |   5 +-
 .../provenance/analyzer/TestPutHiveStreaming.java  |  12 +-
 .../nifi-cdc-mysql-processors/pom.xml              |   2 +-
 .../nifi-elasticsearch-client-service/pom.xml      |   1 -
 .../java/org/apache/nifi/util/db/JdbcCommon.java   |  13 +-
 .../org/apache/nifi/avro/TestAvroTypeUtil.java     |  51 ++++
 .../nifi/authorization/FileAuthorizerTest.java     |   8 +
 ...ncryptedRepositoryRecordSerdeFactoryTest.groovy |   4 +-
 ...EncryptedSchemaRepositoryRecordSerdeTest.groovy |   3 +
 .../protocol/impl/SocketProtocolListenerTest.java  |   2 +
 .../okhttp/OkHttpReplicationClient.java            |  29 +-
 .../okhttp/OkHttpReplicationClientTest.groovy      | 141 ++++++++++
 .../heartbeat/TestAbstractHeartbeatMonitor.java    |   2 +
 .../src/test/resources/conf}/keystore.jks          | Bin
 .../src/test/resources/conf}/truststore.jks        | Bin
 .../components/validation/ValidationState.java     |   4 +
 .../nifi/controller/AbstractComponentNode.java     | 103 ++++++-
 .../org/apache/nifi/controller/ComponentNode.java  |   7 +
 .../nifi/controller/PropertyConfiguration.java     |  26 +-
 .../org/apache/nifi/events/BulletinFactory.java    |  32 ++-
 .../java/org/apache/nifi/groups/ProcessGroup.java  |   5 +-
 .../apache/nifi/parameter/ParameterContext.java    |   9 +-
 .../org/apache/nifi/parameter/ParameterUpdate.java |  19 +-
 .../nifi/controller/TestAbstractComponentNode.java | 123 ++++++++-
 .../controller/service/ServiceStateTransition.java |  16 --
 .../service/StandardControllerServiceNode.java     |   9 +-
 .../apache/nifi/groups/StandardProcessGroup.java   | 128 +++++++--
 .../nifi/parameter/StandardParameterContext.java   |  23 +-
 .../nifi/parameter/StandardParameterUpdate.java    |  83 ++++++
 .../EncryptedFileSystemRepositoryTest.groovy       |   3 +
 ...cryptedSequentialAccessWriteAheadLogTest.groovy |   3 +
 .../repository/TestFileSystemRepository.java       |  11 +-
 .../repository/TestRocksDBFlowFileRepository.java  |  14 +-
 .../controller/service/mock/MockProcessGroup.java  |   3 +-
 .../zookeeper/TestZooKeeperStateProvider.java      |  10 +-
 .../nifi-framework-nar-loading-utils/pom.xml       |   6 +
 .../org/apache/nifi/nar/AbstractTestNarLoader.java |   8 +
 .../java/org/apache/nifi/nar/TestNarLoader.java    |   8 +
 .../AESSensitivePropertyProviderFactoryTest.groovy |   3 +
 .../NiFiPropertiesLoaderGroovyTest.groovy          |   3 +
 .../ProtectedNiFiPropertiesGroovyTest.groovy       |   3 +
 .../nifi/remote/StandardRemoteGroupPort.java       |  13 +-
 .../nifi-framework/nifi-stateless/pom.xml          |   1 -
 .../stateless/core/StatelessRemoteOutputPort.java  |   8 +-
 .../apache/nifi/web/api/DataTransferResource.java  |  19 +-
 .../nifi/web/security/knox/KnoxServiceTest.java    |   8 +
 .../groovyx/ExecuteGroovyScriptTest.java           |   3 +
 .../apache/nifi/processors/hadoop/GetHDFSTest.java |   8 +
 .../nifi/processors/hadoop/MoveHDFSTest.java       |   8 +
 .../processors/hive/TestPutHive3Streaming.java     |   8 +
 .../org/apache/nifi/processors/orc/PutORCTest.java |   5 +-
 nifi-nar-bundles/nifi-hive-bundle/pom.xml          |   2 +-
 ...ion.java => IJMSConnectionFactoryProvider.java} |   7 +-
 .../cf/JMSConnectionFactoryProviderDefinition.java |  25 +-
 .../nifi/jms/cf/JMSConnectionFactoryHandler.java   | 248 +++++++++++++++++
 .../jms/cf/JMSConnectionFactoryProperties.java     | 113 ++++++++
 .../nifi/jms/cf/JMSConnectionFactoryProvider.java  | 307 ++-------------------
 .../jms/cf/JndiJmsConnectionFactoryHandler.java    | 154 +++++++++++
 .../jms/cf/JndiJmsConnectionFactoryProperties.java | 114 ++++++++
 .../jms/cf/JndiJmsConnectionFactoryProvider.java   | 191 ++-----------
 .../nifi/jms/processors/AbstractJMSProcessor.java  | 221 +++++++++++----
 .../org/apache/nifi/jms/processors/ConsumeJMS.java |  34 ++-
 .../org/apache/nifi/jms/processors/PublishJMS.java |  65 +++++
 .../additionalDetails.html                         |  19 +-
 .../additionalDetails.html                         |  23 +-
 .../additionalDetails.html                         |  25 +-
 .../additionalDetails.html                         |  25 +-
 ...ava => JMSConnectionFactoryHandlerForTest.java} |  23 +-
 .../cf/JMSConnectionFactoryProviderForTest.java    |  20 +-
 .../jms/cf/JMSConnectionFactoryProviderTest.java   | 201 +++++++-------
 .../jms/processors/ConnectionFactoryConfigIT.java  | 176 ++++++++++++
 .../ConnectionFactoryConfigValidatorTest.java      | 145 ++++++++++
 .../apache/nifi/jms/processors/ConsumeJMSIT.java   |  51 ++++
 .../apache/nifi/jms/processors/PublishJMSIT.java   |  25 ++
 .../jolt/record/TestJoltTransformRecord.java       |   9 +
 .../nifi-kite-bundle/nifi-kite-processors/pom.xml  |   2 +-
 .../processors/kite/TestCSVToAvroProcessor.java    |   7 +
 .../processors/kite/TestConfigurationProperty.java |   8 +
 .../nifi/processors/kite/TestInferAvroSchema.java  |   8 +
 .../processors/mongodb/AbstractMongoProcessor.java |   4 +-
 .../nifi/processors/mongodb/PutMongoRecord.java    |   3 +-
 .../apache/nifi/processors/mqtt/ConsumeMQTT.java   |  22 +-
 .../mqtt/common/AbstractMQTTProcessor.java         |  13 +-
 .../nifi/processors/mqtt/TestPublishMQTT.java      |   1 -
 .../org/apache/nifi/parquet/TestParquetReader.java |   8 +
 .../nifi/processors/parquet/FetchParquetTest.java  |   8 +
 .../nifi/processors/parquet/PutParquetTest.java    |   5 +-
 .../poi/ConvertExcelToCSVProcessorTest.java        |   8 +
 .../index/lucene/TestLuceneEventIndex.java         |  26 +-
 .../reporting/SiteToSiteBulletinReportingTask.java |   1 +
 .../additionalDetails.html                         |   1 +
 .../src/main/resources/schema-bulletins.avsc       |   1 +
 .../TestSiteToSiteBulletinReportingTask.java       |   5 +-
 .../TestSiteToSiteMetricsReportingTask.java        |   8 +
 .../sql/bulletins/BulletinEnumerator.java          |   1 +
 .../reporting/sql/bulletins/BulletinTable.java     |   2 +
 .../additionalDetails.html                         |   1 +
 .../nifi-standard-processors/pom.xml               |   4 +
 .../standard/AbstractDatabaseFetchProcessor.java   |   4 +-
 .../processors/standard/FetchFileTransfer.java     |  18 +-
 .../nifi/processors/standard/GetFileTransfer.java  |  10 +-
 .../nifi/processors/standard/InvokeHTTP.java       |  16 +-
 .../nifi/processors/standard/LookupRecord.java     | 126 +++++++--
 .../nifi/processors/standard/PutFileTransfer.java  |   5 +-
 .../nifi/processors/standard/ReplaceText.java      |  37 ++-
 .../nifi/processors/standard/util/FTPTransfer.java |  62 +++--
 .../processors/standard/util/FileTransfer.java     |  10 +-
 .../processors/standard/util/SFTPTransfer.java     |  58 ++--
 .../additionalDetails.html                         | 215 +++++++++++++++
 .../processors/standard/TestConvertRecord.java     |   9 +
 .../processors/standard/TestEncryptContent.java    |   7 +
 .../standard/TestExecuteStreamCommand.java         |   3 +
 .../nifi/processors/standard/TestFetchFile.java    |   8 +
 .../processors/standard/TestFetchFileTransfer.java |  31 +--
 .../nifi/processors/standard/TestForkRecord.java   |   9 +
 .../nifi/processors/standard/TestInvokeHTTP.java   |  46 ++-
 .../processors/standard/TestInvokeHttpSSL.java     |   8 +-
 .../standard/TestInvokeHttpTwoWaySSL.java          |   3 +
 .../nifi/processors/standard/TestListFile.java     |   8 +
 .../nifi/processors/standard/TestListenHTTP.java   |  10 +-
 .../nifi/processors/standard/TestListenSyslog.java |   9 +
 .../nifi/processors/standard/TestLookupRecord.java |  89 ++++++
 .../processors/standard/TestMonitorActivity.java   |  61 ++--
 .../nifi/processors/standard/TestPutFile.java      |   8 +
 .../nifi/processors/standard/TestPutSQL.java       |  26 +-
 .../nifi/processors/standard/TestUpdateRecord.java |   9 +
 .../processors/standard/TestValidateRecord.java    |  47 ++++
 .../relp/handler/TestRELPSocketChannelHandler.java |   4 +-
 .../util/ITestSFTPTransferWithSSHTestServer.java   |  23 --
 .../standard/util/TestInvokeHttpCommon.java        |   1 -
 .../processors/standard/util/TestPutTCPCommon.java |   8 +
 .../util/crypto/OpenPGPKeyBasedEncryptorTest.java  |   3 +
 .../crypto/OpenPGPPasswordBasedEncryptorTest.java  |   3 +
 .../TestLookupRecord/lookup-array-input.json       |  29 ++
 .../TestLookupRecord/lookup-array-output.json      |   1 +
 .../TestValidateRecord/int-maps-data.json          |  27 ++
 .../TestValidateRecord/int-maps-schema.avsc        |  44 +++
 .../java/org/apache/nifi/dbcp/DBCPServiceTest.java |  88 ++----
 .../nifi/json/AbstractJsonRowRecordReader.java     |  24 +-
 .../apache/nifi/json/JsonPathRowRecordReader.java  |   4 +-
 .../org/apache/nifi/avro/TestWriteAvroResult.java  |   1 -
 .../nifi/json/TestJsonPathRowRecordReader.java     |   4 +
 .../org/apache/nifi/json/TestWriteJsonResult.java  |   5 +-
 .../src/test/resources/json/timestamp.json         |   2 +-
 ...ngWindow.java => AttributeRollingWindowIT.java} |   2 +-
 .../admin/configmigrator/ConfigMigratorSpec.groovy |   7 +-
 .../admin/filemanager/FileManagerToolSpec.groovy   |   6 +
 nifi-toolkit/nifi-toolkit-assembly/pom.xml         |  53 ++++
 .../command/registry/flow/ImportFlowVersion.java   |   6 +-
 .../apache/nifi/toolkit/cli/TestCLICompleter.java  |   3 +
 .../toolkit/cli/impl/result/TestBucketsResult.java |   8 +
 .../cli/impl/result/TestRegistryClientResult.java  |   8 +
 .../TestVersionedFlowSnapshotMetadataResult.java   |   8 +
 .../cli/impl/result/TestVersionedFlowsResult.java  |   8 +
 .../impl/result/writer/TestDynamicTableWriter.java |   8 +
 .../properties/ConfigEncryptionToolTest.groovy     |  20 +-
 nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml    |   6 +
 .../flowfile/TestRepairCorruptedFileEndings.java   |   8 +
 .../apache/nifi/toolkit/s2s/SiteToSiteCliMain.java |   3 -
 .../nifi/toolkit/s2s/SiteToSiteReceiver.java       |   4 -
 pom.xml                                            |  11 +-
 196 files changed, 4498 insertions(+), 1448 deletions(-)
 delete mode 100644 nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoContentException.java
 delete mode 100644 nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoValidPeerException.java
 rename nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/{TestPutRecord.java => PutRecordIT.java} (98%)
 rename nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/{TestScanAccumulo.java => ScanAccumuloIT.java} (98%)
 copy nifi-nar-bundles/{nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/resources/certs => nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf}/keystore.jks (100%)
 copy {nifi-toolkit/nifi-toolkit-tls/src/test/resources/localhost => nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf}/truststore.jks (100%)
 copy nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterReference.java => nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterUpdate.java (60%)
 create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterUpdate.java
 copy nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/{JMSConnectionFactoryProviderDefinition.java => IJMSConnectionFactoryProvider.java} (87%)
 create mode 100644 nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java
 create mode 100644 nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProperties.java
 create mode 100644 nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryHandler.java
 create mode 100644 nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProperties.java
 copy nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/{JMSConnectionFactoryProviderForTest.java => JMSConnectionFactoryHandlerForTest.java} (59%)
 create mode 100644 nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigIT.java
 create mode 100644 nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigValidatorTest.java
 create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LookupRecord/additionalDetails.html
 create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-input.json
 create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-output.json
 create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-data.json
 create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-schema.avsc
 rename nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/test/java/org/apache/nifi/processors/stateful/analysis/{TestAttributeRollingWindow.java => AttributeRollingWindowIT.java} (99%)


[nifi] 35/47: NIFI-7248: Atlas reporting task handles PutHive3Streaming

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit c44204d77c95a18a140c1c0d6077c3c0544954b1
Author: Peter Turcsanyi <tu...@apache.org>
AuthorDate: Thu Mar 12 09:31:45 2020 +0100

    NIFI-7248: Atlas reporting task handles PutHive3Streaming
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4138.
---
 .../nifi/atlas/provenance/analyzer/PutHiveStreaming.java     |  5 ++++-
 .../nifi/atlas/provenance/analyzer/TestPutHiveStreaming.java | 12 ++++++++++--
 2 files changed, 14 insertions(+), 3 deletions(-)

diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/analyzer/PutHiveStreaming.java b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/analyzer/PutHiveStreaming.java
index 78c37ea..de705da 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/analyzer/PutHiveStreaming.java
+++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/analyzer/PutHiveStreaming.java
@@ -37,6 +37,9 @@ public class PutHiveStreaming extends AbstractHiveAnalyzer {
 
     @Override
     public DataSetRefs analyze(AnalysisContext context, ProvenanceEventRecord event) {
+        if (event.getTransitUri() == null) {
+            return null;
+        }
 
         final URI uri = parseUri(event.getTransitUri());
         final String clusterName = context.getClusterResolver().fromHostNames(uri.getHost());
@@ -55,6 +58,6 @@ public class PutHiveStreaming extends AbstractHiveAnalyzer {
 
     @Override
     public String targetComponentTypePattern() {
-        return "^PutHiveStreaming$";
+        return "^PutHive(3)?Streaming$";
     }
 }
diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/test/java/org/apache/nifi/atlas/provenance/analyzer/TestPutHiveStreaming.java b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/test/java/org/apache/nifi/atlas/provenance/analyzer/TestPutHiveStreaming.java
index 5184025..606f6d5 100644
--- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/test/java/org/apache/nifi/atlas/provenance/analyzer/TestPutHiveStreaming.java
+++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/test/java/org/apache/nifi/atlas/provenance/analyzer/TestPutHiveStreaming.java
@@ -38,8 +38,16 @@ import static org.mockito.Mockito.when;
 public class TestPutHiveStreaming {
 
     @Test
-    public void testTableLineage() {
-        final String processorName = "PutHiveStreaming";
+    public void testTableLineageHive1() {
+        testTableLineage("PutHiveStreaming");
+    }
+
+    @Test
+    public void testTableLineageHive3() {
+        testTableLineage("PutHive3Streaming");
+    }
+
+    private void testTableLineage(String processorName) {
         final String transitUri = "thrift://0.example.com:9083";
         final ProvenanceEventRecord record = Mockito.mock(ProvenanceEventRecord.class);
         when(record.getComponentType()).thenReturn(processorName);


[nifi] 13/47: NIFI-7227 Fixed typo in Global Access Policy table (#4112)

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 65ffd45650fc502c1763dc780dc851f819567fca
Author: Andy LoPresto <al...@users.noreply.github.com>
AuthorDate: Wed Mar 4 12:59:05 2020 -0800

    NIFI-7227 Fixed typo in Global Access Policy table (#4112)
    
    Co-authored-by: spius <57...@users.noreply.github.com>
    
    Signed-off-by: Andy LoPresto <al...@apache.org>
---
 nifi-docs/src/main/asciidoc/administration-guide.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 7093fbd..5c90725 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -1104,7 +1104,7 @@ Global access policies govern the following system level authorizations:
 |Policy |Privilege |Global Menu Selection |Resource Descriptor
 
 |view the UI
-|Allow users to view the UI
+|Allows users to view the UI
 |N/A
 |`/flow`
 


[nifi] 27/47: NIFI-7239: Upgrade the Hive 3 bundle to use Apache Hive 3.1.2

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 716e32e5bfaa8dc6f6d4d09356d0dd6fed5fa4d4
Author: Peter Turcsanyi <tu...@apache.org>
AuthorDate: Tue Mar 10 19:26:05 2020 +0100

    NIFI-7239: Upgrade the Hive 3 bundle to use Apache Hive 3.1.2
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4129.
---
 nifi-nar-bundles/nifi-hive-bundle/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-hive-bundle/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
index ce482aa..4b62a50 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
@@ -53,7 +53,7 @@
         <hive11.hadoop.version>2.6.2</hive11.hadoop.version>
         <hive.version>1.2.1</hive.version>
         <hive.hadoop.version>2.6.2</hive.hadoop.version>
-        <hive3.version>3.1.0</hive3.version>
+        <hive3.version>3.1.2</hive3.version>
         <hive3.hadoop.version>3.1.0</hive3.hadoop.version>
     </properties>
 


[nifi] 41/47: NIFI-7250 activate user.timezone appropriate to each region NIFI-7250 fix a test which appears brittle at least on windows builds on slow environments NIFI-7250 activated a timezone run for AU Australia/Melbourne which exposed a poor magic number and needless assertion but interesting results worth keeping

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 24ece001cd2f551b2dd8aaecb639ff356324858b
Author: Joe Witt <jo...@apache.org>
AuthorDate: Thu Mar 12 15:13:58 2020 -0400

    NIFI-7250 activate user.timezone appropriate to each region
    NIFI-7250 fix a test which appears brittle at least on windows builds on slow environments
    NIFI-7250 activated a timezone run for AU Australia/Melbourne which exposed a poor magic number and needless assertion but interesting results worth keeping
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4140.
---
 .github/workflows/ci-workflow.yml                                | 8 ++++----
 .../org/apache/nifi/processors/standard/TestListenSyslog.java    | 9 +++++++++
 .../src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java  | 1 -
 3 files changed, 13 insertions(+), 5 deletions(-)

diff --git a/.github/workflows/ci-workflow.yml b/.github/workflows/ci-workflow.yml
index d8800fb..0f86629 100644
--- a/.github/workflows/ci-workflow.yml
+++ b/.github/workflows/ci-workflow.yml
@@ -53,7 +53,7 @@ jobs:
         java-version: 1.8
     - name: Build with Maven
       env:
-        MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=fr -Duser.region=FR"
+        MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=fr -Duser.region=FR -Duser.timezone=Europe/Paris"
       run: |
         mvn -version
         mvn -T 1C install -B -Pcontrib-check,include-grpc -Ddir-only -ntp -ff
@@ -109,7 +109,7 @@ jobs:
           java-version: 11
       - name: Build with Maven
         env:
-          MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=en -Duser.region=US"
+          MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=en -Duser.region=AU -Duser.timezone=Australia/Melbourne"
         run: |
           mvn -version
           mvn -T 1C install -B -Pcontrib-check,include-grpc -Ddir-only -ntp -ff
@@ -165,7 +165,7 @@ jobs:
           java-version: 1.8
       - name: Build with Maven
         env:
-          MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=ja -Duser.region=JP"
+          MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=ja -Duser.region=JP -Duser.timezone=Asia/Tokyo"
         run: |
           mvn -version
           mvn -T 1C install -B -Pcontrib-check,include-grpc -Ddir-only -ntp -ff
@@ -217,7 +217,7 @@ jobs:
           java-version: 1.8
       - name: Build with Maven
         env:
-          MAVEN_OPTS: -Xmx2g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN
+          MAVEN_OPTS: -Xmx2g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=en -Duser.region=US -Duser.timezone=US/Pacific"
         run: |
           mvn -version
           mvn -T 1C install -B -Ddir-only -ntp -ff
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java
index 7fddc1b..c045c75 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.standard;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.ExpressionLanguageScope;
@@ -34,6 +35,8 @@ import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -66,6 +69,12 @@ public class TestListenSyslog {
     static final String VALID_MESSAGE_TCP = "<" + PRI + ">" + TIME + " " + HOST + " " + BODY + "\n";
     static final String INVALID_MESSAGE = "this is not valid\n";
 
+    @BeforeClass
+    public static void setupBeforeClass() throws Exception {
+        //These tests are unreliable on slow windows builds for some reason
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Test
     public void testBatching() throws IOException, InterruptedException {
         final ListenSyslog proc = new ListenSyslog();
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java
index d3e5f6c..7a35ba5 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java
@@ -194,7 +194,6 @@ public abstract class TestWriteAvroResult {
             assertEquals(millisSinceMidnight * 1000L, avroRecord.get("timeMicros"));
             assertEquals(timeLong, avroRecord.get("timestampMillis"));
             assertEquals(timeLong * 1000L, avroRecord.get("timestampMicros"));
-            assertEquals(17260, avroRecord.get("date"));
             // Double value will be converted into logical decimal if Avro schema is defined as logical decimal.
             final Schema decimalSchema = schema.getField("decimal").schema();
             final LogicalType logicalType = decimalSchema.getLogicalType() != null


[nifi] 21/47: NIFI-7224 Protecting against possible NPE in ImportFlowVersion command in CLI

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit b7cbd23f7c22de6453e6f83c5f492f74e17b3238
Author: Bryan Bende <bb...@apache.org>
AuthorDate: Fri Mar 6 16:08:09 2020 -0500

    NIFI-7224 Protecting against possible NPE in ImportFlowVersion command in CLI
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4121.
---
 .../toolkit/cli/impl/command/registry/flow/ImportFlowVersion.java   | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportFlowVersion.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportFlowVersion.java
index d9a3530..c02f38e 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportFlowVersion.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportFlowVersion.java
@@ -89,7 +89,11 @@ public class ImportFlowVersion extends AbstractNiFiRegistryCommand<StringResult>
         metadata.setBucketIdentifier(versionedFlow.getBucketIdentifier());
         metadata.setFlowIdentifier(flowId);
         metadata.setVersion(version);
-        metadata.setComments(deserializedSnapshot.getSnapshotMetadata().getComments());
+
+        final VersionedFlowSnapshotMetadata deserializedSnapshotMetadata = deserializedSnapshot.getSnapshotMetadata();
+        if (deserializedSnapshotMetadata != null) {
+            metadata.setComments(deserializedSnapshotMetadata.getComments());
+        }
 
         // create a new snapshot using the new metadata and the contents from the deserialized snapshot
         final VersionedFlowSnapshot snapshot = new VersionedFlowSnapshot();


[nifi] 09/47: NIFI-7218 Fixed typo in Overview docs. (#4107)

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 089ca0216054fe718e7c3a571f4c3497cbc7ff81
Author: Andy LoPresto <al...@users.noreply.github.com>
AuthorDate: Tue Mar 3 15:47:42 2020 -0800

    NIFI-7218 Fixed typo in Overview docs. (#4107)
---
 nifi-docs/src/main/asciidoc/overview.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/nifi-docs/src/main/asciidoc/overview.adoc b/nifi-docs/src/main/asciidoc/overview.adoc
index 594d9a3..78ab665 100644
--- a/nifi-docs/src/main/asciidoc/overview.adoc
+++ b/nifi-docs/src/main/asciidoc/overview.adoc
@@ -20,7 +20,7 @@ Apache NiFi Team <de...@nifi.apache.org>
 :linkattrs:
 
 == What is Apache NiFi?
-Put simply NiFi was built to automate the flow of data between systems.  While
+Put simply, NiFi was built to automate the flow of data between systems.  While
 the term 'dataflow' is used in a variety of contexts, we use it here
 to mean the automated and managed flow of information between systems.  This
 problem space has been around ever since enterprises had more than one system,


[nifi] 06/47: NIFI-7205 NIFI-7206

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 024d727ae0c0b4b19cc51b7c10e81781cbee1658
Author: Joe Witt <jo...@apache.org>
AuthorDate: Wed Feb 26 11:55:21 2020 -0800

    NIFI-7205 NIFI-7206
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4093.
---
 .../state/providers/zookeeper/TestZooKeeperStateProvider.java  | 10 +++++-----
 .../standard/relp/handler/TestRELPSocketChannelHandler.java    |  4 ++--
 2 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
index 091b13c..cac0cf9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
@@ -45,7 +45,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
     private static final Map<PropertyDescriptor, String> defaultProperties = new HashMap<>();
 
     static {
-        defaultProperties.put(ZooKeeperStateProvider.SESSION_TIMEOUT, "3 secs");
+        defaultProperties.put(ZooKeeperStateProvider.SESSION_TIMEOUT, "15 secs");
         defaultProperties.put(ZooKeeperStateProvider.ROOT_NODE, "/nifi/team1/testing");
         defaultProperties.put(ZooKeeperStateProvider.ACCESS_CONTROL, ZooKeeperStateProvider.OPEN_TO_WORLD.getValue());
     }
@@ -131,7 +131,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
         return provider;
     }
 
-    @Test(timeout = 20000)
+    @Test(timeout = 30000)
     public void testStateTooLargeExceptionThrownOnSetState() throws InterruptedException {
         final Map<String, String> state = new HashMap<>();
         final StringBuilder sb = new StringBuilder();
@@ -157,7 +157,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
                 // If we attempt to interact with the server too quickly, we will get a
                 // ZooKeeper ConnectionLoss Exception, which the provider wraps in an IOException.
                 // We will wait 1 second in this case and try again. The test will timeout if this
-                // does not succeeed within 20 seconds.
+                // does not succeeed within 30 seconds.
                 Thread.sleep(1000L);
             } catch (final Exception e) {
                 e.printStackTrace();
@@ -166,7 +166,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
         }
     }
 
-    @Test(timeout = 20000)
+    @Test(timeout = 30000)
     public void testStateTooLargeExceptionThrownOnReplace() throws IOException, InterruptedException {
         final Map<String, String> state = new HashMap<>();
         final StringBuilder sb = new StringBuilder();
@@ -192,7 +192,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
                 // If we attempt to interact with the server too quickly, we will get a
                 // ZooKeeper ConnectionLoss Exception, which the provider wraps in an IOException.
                 // We will wait 1 second in this case and try again. The test will timeout if this
-                // does not succeeed within 20 seconds.
+                // does not succeeed within 30 seconds.
                 Thread.sleep(1000L);
             }
         }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/relp/handler/TestRELPSocketChannelHandler.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/relp/handler/TestRELPSocketChannelHandler.java
index 4ac1fb7..5bfaca7 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/relp/handler/TestRELPSocketChannelHandler.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/relp/handler/TestRELPSocketChannelHandler.java
@@ -153,8 +153,8 @@ public class TestRELPSocketChannelHandler {
                 }
             }
 
-            // wait up to 10 seconds to verify the responses
-            long timeout = 10000;
+            // wait up to 25 seconds to verify the responses
+            long timeout = 25000;
             long startTime = System.currentTimeMillis();
             while (events.size() < messages.size() && (System.currentTimeMillis() - startTime < timeout)) {
                 Thread.sleep(100);


[nifi] 18/47: NIFI-7191 Conditionally disable docker integration tests

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 20c086c4fa96682e6757299dc9e9c5e4a018a768
Author: Kevin Doran <kd...@apache.org>
AuthorDate: Mon Feb 24 13:51:17 2020 -0500

    NIFI-7191 Conditionally disable docker integration tests
    
    Honor Maven properties skipTests and maven.test.skip for the
    shell script integration tests that verifiy docker images.
---
 nifi-docker/dockerhub/pom.xml              | 45 +++++++++++++++++++++++++
 nifi-docker/dockermaven/pom.xml            | 45 +++++++++++++++++++++++++
 nifi-toolkit/nifi-toolkit-assembly/pom.xml | 53 ++++++++++++++++++++++++++++++
 3 files changed, 143 insertions(+)

diff --git a/nifi-docker/dockerhub/pom.xml b/nifi-docker/dockerhub/pom.xml
index c8e98fe..f7db33f 100644
--- a/nifi-docker/dockerhub/pom.xml
+++ b/nifi-docker/dockerhub/pom.xml
@@ -71,6 +71,51 @@
                 </plugins>
             </build>
         </profile>
+        <profile>
+            <id>docker-skip-tests</id>
+            <activation>
+                <property>
+                    <name>skipTests</name>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <executions>
+                            <execution>
+                                <id>Docker integration tests</id>
+                                <phase>none</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <id>docker-test-skip-test</id>
+            <activation>
+                <property>
+                    <name>maven.test.skip</name>
+                    <value>true</value>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <executions>
+                            <execution>
+                                <id>Docker integration tests</id>
+                                <phase>none</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
     </profiles>
 
 </project>
\ No newline at end of file
diff --git a/nifi-docker/dockermaven/pom.xml b/nifi-docker/dockermaven/pom.xml
index a575fcd..52640cb 100644
--- a/nifi-docker/dockermaven/pom.xml
+++ b/nifi-docker/dockermaven/pom.xml
@@ -111,6 +111,51 @@
                 </plugins>
             </build>
         </profile>
+        <profile>
+            <id>docker-skip-tests</id>
+            <activation>
+                <property>
+                    <name>skipTests</name>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <executions>
+                            <execution>
+                                <id>Docker integration tests</id>
+                                <phase>none</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <id>docker-test-skip-test</id>
+            <activation>
+                <property>
+                    <name>maven.test.skip</name>
+                    <value>true</value>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <executions>
+                            <execution>
+                                <id>Docker integration tests</id>
+                                <phase>none</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
     </profiles>
 
 </project>
\ No newline at end of file
diff --git a/nifi-toolkit/nifi-toolkit-assembly/pom.xml b/nifi-toolkit/nifi-toolkit-assembly/pom.xml
index 565b018..ebd7e0f 100644
--- a/nifi-toolkit/nifi-toolkit-assembly/pom.xml
+++ b/nifi-toolkit/nifi-toolkit-assembly/pom.xml
@@ -255,5 +255,58 @@ language governing permissions and limitations under the License. -->
                 </plugins>
             </build>
         </profile>
+        <profile>
+            <id>docker-skip-tests</id>
+            <activation>
+                <property>
+                    <name>skipTests</name>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <executions>
+                            <execution>
+                                <id>Docker integration tests - exit codes</id>
+                                <phase>none</phase>
+                            </execution>
+                            <execution>
+                                <id>Docker integration tests - tls-toolkit</id>
+                                <phase>none</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <id>docker-test-skip-test</id>
+            <activation>
+                <property>
+                    <name>maven.test.skip</name>
+                    <value>true</value>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <executions>
+                            <execution>
+                                <id>Docker integration tests - exit codes</id>
+                                <phase>none</phase>
+                            </execution>
+                            <execution>
+                                <id>Docker integration tests - tls-toolkit</id>
+                                <phase>none</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
     </profiles>
 </project>


[nifi] 33/47: NIFI-7119 Implement boundary checking for Argon2 cost parameters (#4111)

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 950873c3870acf48d40444b039ebeb692b041021
Author: M Tien <56...@users.noreply.github.com>
AuthorDate: Wed Mar 11 15:51:15 2020 -0700

    NIFI-7119 Implement boundary checking for Argon2 cost parameters (#4111)
    
    * NIFI-7119 Implemented parameter boundary enforcement for Argon2SecureHasher constructor.
    Added unit tests for validating each parameter check.
    
    * NIFI-7119 Refactored parameter validations. Added more test sizes to boundary checkers. Changed logger severity to error and added bounds to messages.
    
    * NIFI-7119 Refactored Argon2 parameter data types to handle unsigned integer boundary values.
    Updated unit tests.
    
    Co-authored-by: Andy LoPresto <al...@apache.org>
    
    Signed-off-by: Andy LoPresto <al...@apache.org>
---
 .../security/util/crypto/Argon2SecureHasher.java   | 138 ++++++++++++++++--
 .../util/crypto/Argon2SecureHasherTest.groovy      | 156 +++++++++++++++++++++
 2 files changed, 284 insertions(+), 10 deletions(-)

diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
index 22e618c..7ae6dc3 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
@@ -46,17 +46,28 @@ public class Argon2SecureHasher implements SecureHasher {
     private static final int DEFAULT_PARALLELISM = 1;
     private static final int DEFAULT_MEMORY = 1 << 12;
     private static final int DEFAULT_ITERATIONS = 3;
+    private static final int DEFAULT_SALT_LENGTH = 16;
+    private static final int MIN_MEMORY_SIZE_KB = 8;
+    private static final int MIN_PARALLELISM = 1;
+    private static final long MAX_PARALLELISM = Math.round(Math.pow(2, 24)) - 1;
+    private static final int MIN_HASH_LENGTH = 4;
+    private static final int MIN_ITERATIONS = 1;
+    private static final int MIN_SALT_LENGTH = 8;
 
-    private final int hashLength;
-    private final int memory;
+    // Using Integer vs. int to allow for unsigned 32b (values can exceed Integer.MAX_VALUE)
+    private final Integer hashLength;
+    private final Integer memory;
     private final int parallelism;
-    private final int iterations;
-    private final int saltLength;
+    private final Integer iterations;
+    private final Integer saltLength;
 
-    private final boolean usingStaticSalt;
+    private boolean usingStaticSalt;
 
     // A 16 byte salt (nonce) is recommended for password hashing
-    private static final byte[] staticSalt = "NiFi Static Salt".getBytes(StandardCharsets.UTF_8);
+    private static final byte[] STATIC_SALT = "NiFi Static Salt".getBytes(StandardCharsets.UTF_8);
+
+    // Upper boundary for several cost parameters
+    private static final long UPPER_BOUNDARY = Math.round(Math.pow(2, 32)) - 1;
 
     /**
      * Instantiates an Argon2 secure hasher using the default cost parameters
@@ -72,19 +83,21 @@ public class Argon2SecureHasher implements SecureHasher {
     /**
      * Instantiates an Argon2 secure hasher using the provided cost parameters. A static
      * {@link #DEFAULT_SALT_LENGTH} byte salt will be generated on every hash request.
+     * {@link Integer} is used instead of {@code int} for parameters which have a max value of {@code 2^32 - 1} to allow for unsigned integers exceeding {@link Integer#MAX_VALUE}.
      *
      * @param hashLength  the output length in bytes ({@code 4 to 2^32 - 1})
      * @param memory      the integer number of KB used ({@code 8p to 2^32 - 1})
      * @param parallelism degree of parallelism ({@code 1 to 2^24 - 1})
      * @param iterations  number of iterations ({@code 1 to 2^32 - 1})
      */
-    public Argon2SecureHasher(int hashLength, int memory, int parallelism, int iterations) {
+    public Argon2SecureHasher(Integer hashLength, Integer memory, int parallelism, Integer iterations) {
         this(hashLength, memory, parallelism, iterations, 0);
     }
 
     /**
      * Instantiates an Argon2 secure hasher using the provided cost parameters. A unique
      * salt of the specified length will be generated on every hash request.
+     * {@link Integer} is used instead of {@code int} for parameters which have a max value of {@code 2^32 - 1} to allow for unsigned integers exceeding {@link Integer#MAX_VALUE}.
      *
      * @param hashLength  the output length in bytes ({@code 4 to 2^32 - 1})
      * @param memory      the integer number of KB used ({@code 8p to 2^32 - 1})
@@ -92,15 +105,51 @@ public class Argon2SecureHasher implements SecureHasher {
      * @param iterations  number of iterations ({@code 1 to 2^32 - 1})
      * @param saltLength  the salt length in bytes {@code 8 to 2^32 - 1})
      */
-    public Argon2SecureHasher(int hashLength, int memory, int parallelism, int iterations, int saltLength) {
-        // TODO: Implement boundary checking
+    public Argon2SecureHasher(Integer hashLength, Integer memory, int parallelism, Integer iterations, Integer saltLength) {
+
+        validateParameters(hashLength, memory, parallelism, iterations, saltLength);
+
         this.hashLength = hashLength;
         this.memory = memory;
         this.parallelism = parallelism;
         this.iterations = iterations;
 
         this.saltLength = saltLength;
+    }
+
+    /**
+     * Enforces valid Argon2 secure hasher cost parameters are provided.
+     *
+     * @param hashLength  the output length in bytes ({@code 4 to 2^32 - 1})
+     * @param memory      the integer number of KB used ({@code 8p to 2^32 - 1})
+     * @param parallelism degree of parallelism ({@code 1 to 2^24 - 1})
+     * @param iterations  number of iterations ({@code 1 to 2^32 - 1})
+     * @param saltLength  the salt length in bytes {@code 8 to 2^32 - 1})
+     */
+    private void validateParameters(Integer hashLength, Integer memory, int parallelism, Integer iterations, Integer saltLength) {
+
+        if (!isHashLengthValid(hashLength)) {
+            logger.error("The provided hash length {} is outside the boundary of 4 to 2^32 - 1.", hashLength);
+            throw new IllegalArgumentException("Invalid hash length is not within the hashLength boundary.");
+        }
+        if (!isMemorySizeValid(memory)) {
+            logger.error("The provided memory size {} KiB is outside the boundary of 8p to 2^32 - 1.", memory);
+            throw new IllegalArgumentException("Invalid memory size is not within the memory boundary.");
+        }
+        if (!isParallelismValid(parallelism)) {
+            logger.error("The provided parallelization factor {} is outside the boundary of 1 to 2^24 - 1.", parallelism);
+            throw new IllegalArgumentException("Invalid parallelization factor exceeds the parallelism boundary.");
+        }
+        if (!isIterationsValid(iterations)) {
+            logger.error("The iteration count {} is outside the boundary of 1 to 2^32 - 1.", iterations);
+            throw new IllegalArgumentException("Invalid iteration count exceeds the iterations boundary.");
+        }
+
         if (saltLength > 0) {
+            if (!isSaltLengthValid(saltLength)) {
+                logger.error("The salt length {} is outside the boundary of 8 to 2^32 - 1.", saltLength);
+                throw new IllegalArgumentException("Invalid salt length exceeds the saltLength boundary.");
+            }
             this.usingStaticSalt = false;
         } else {
             this.usingStaticSalt = true;
@@ -126,7 +175,7 @@ public class Argon2SecureHasher implements SecureHasher {
      */
     byte[] getSalt() {
         if (isUsingStaticSalt()) {
-            return staticSalt;
+            return STATIC_SALT;
         } else {
             SecureRandom sr = new SecureRandom();
             byte[] salt = new byte[saltLength];
@@ -136,6 +185,75 @@ public class Argon2SecureHasher implements SecureHasher {
     }
 
     /**
+     * Returns whether the provided hash length is within boundaries. The lower bound >= 4 and the
+     * upper bound <= 2^32 - 1.
+     * @param hashLength the output length in bytes
+     * @return true if hashLength is within boundaries
+     */
+    public static boolean isHashLengthValid(Integer hashLength) {
+        if (hashLength < DEFAULT_HASH_LENGTH) {
+            logger.warn("The provided hash length {} is below the recommended minimum {}.", hashLength, DEFAULT_HASH_LENGTH);
+        }
+        return hashLength >= MIN_HASH_LENGTH && hashLength <= UPPER_BOUNDARY;
+    }
+
+    /**
+     * Returns whether the provided memory size is within boundaries. The lower bound >= 8 and the
+     * upper bound <= 2^32 - 1.
+     * @param memory the integer number of KiB used
+     * @return true if memory is within boundaries
+     */
+    public static boolean isMemorySizeValid(Integer memory) {
+        if (memory < DEFAULT_MEMORY) {
+            logger.warn("The provided memory size {} KiB is below the recommended minimum {} KiB.", memory, DEFAULT_MEMORY);
+        }
+        return memory >= MIN_MEMORY_SIZE_KB && memory <= UPPER_BOUNDARY;
+    }
+
+    /**
+     * Returns whether the provided parallelization factor is within boundaries. The lower bound >= 1 and the
+     * upper bound <= 2^24 - 1.
+     * @param parallelism degree of parallelism
+     * @return true if parallelism is within boundaries
+     */
+    public static boolean isParallelismValid(int parallelism) {
+        if (parallelism < DEFAULT_PARALLELISM) {
+            logger.warn("The provided parallelization factor {} is below the recommended minimum {}.", parallelism, DEFAULT_PARALLELISM);
+        }
+        return parallelism >= MIN_PARALLELISM && parallelism <= MAX_PARALLELISM;
+    }
+
+    /**
+     * Returns whether the provided iteration count is within boundaries. The lower bound >= 1 and the
+     * upper bound <= 2^32 - 1.
+     * @param iterations number of iterations
+     * @return true if iterations is within boundaries
+     */
+    public static boolean isIterationsValid(Integer iterations) {
+        if (iterations < DEFAULT_ITERATIONS) {
+            logger.warn("The provided iteration count {} is below the recommended minimum {}.", iterations, DEFAULT_ITERATIONS);
+        }
+        return iterations >= MIN_ITERATIONS && iterations <= UPPER_BOUNDARY;
+    }
+
+    /**
+     * Returns whether the provided salt length (saltLength) is within boundaries. The lower bound >= 8 and the
+     * upper bound <= 2^32 - 1.
+     * @param saltLength the salt length in bytes
+     * @return true if saltLength is within boundaries
+     */
+    public static boolean isSaltLengthValid(Integer saltLength) {
+        if (saltLength == 0) {
+            logger.debug("The provided salt length 0 indicates a static salt of {} bytes", DEFAULT_SALT_LENGTH);
+            return true;
+        }
+        if (saltLength < DEFAULT_SALT_LENGTH) {
+            logger.warn("The provided dynamic salt length {} is below the recommended minimum {}", saltLength, DEFAULT_SALT_LENGTH);
+        }
+        return saltLength >= MIN_SALT_LENGTH && saltLength <= UPPER_BOUNDARY;
+    }
+
+    /**
      * Returns a String representation of {@code Argon2(input)} in hex-encoded format.
      *
      * @param input the input
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.groovy
index e952a70..9a8b1ae 100644
--- a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.groovy
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.groovy
@@ -239,4 +239,160 @@ class Argon2SecureHasherTest extends GroovyTestCase {
         assert resultDurations.min() > MIN_DURATION_NANOS
         assert resultDurations.sum() / testIterations > MIN_DURATION_NANOS
     }
+
+    @Test
+    void testShouldVerifyHashLengthBoundary() throws Exception {
+        // Arrange
+        final int hashLength = 128
+
+        // Act
+        boolean valid = Argon2SecureHasher.isHashLengthValid(hashLength)
+
+        // Assert
+        assert valid
+    }
+
+    @Test
+    void testShouldFailHashLengthBoundary() throws Exception {
+        // Arrange
+        def hashLengths = [-8, 0, 1, 2]
+
+        // Act
+        def results = hashLengths.collect { hashLength ->
+            def isValid = Argon2SecureHasher.isHashLengthValid(hashLength)
+            [hashLength, isValid]
+        }
+
+        // Assert
+        results.each { hashLength, isHashLengthValid ->
+            logger.info("For hashLength value ${hashLength}, hashLength is ${isHashLengthValid ? "valid" : "invalid"}")
+            assert !isHashLengthValid
+        }
+    }
+
+    @Test
+    void testShouldVerifyMemorySizeBoundary() throws Exception {
+        // Arrange
+        final int memory = 2048
+
+        // Act
+        boolean valid = Argon2SecureHasher.isMemorySizeValid(memory)
+
+        // Assert
+        assert valid
+    }
+
+    @Test
+    void testShouldFailMemorySizeBoundary() throws Exception {
+        // Arrange
+        def memorySizes = [-12, 0, 1, 6]
+
+        // Act
+        def results = memorySizes.collect { memory ->
+            def isValid = Argon2SecureHasher.isMemorySizeValid(memory)
+            [memory, isValid]
+        }
+
+        // Assert
+        results.each { memory, isMemorySizeValid ->
+            logger.info("For memory size ${memory}, memory is ${isMemorySizeValid ? "valid" : "invalid"}")
+            assert !isMemorySizeValid
+        }
+    }
+
+    @Test
+    void testShouldVerifyParallelismBoundary() throws Exception {
+        // Arrange
+        final int parallelism = 4
+
+        // Act
+        boolean valid = Argon2SecureHasher.isParallelismValid(parallelism)
+
+        // Assert
+        assert valid
+    }
+
+    @Test
+    void testShouldFailParallelismBoundary() throws Exception {
+        // Arrange
+        def parallelisms = [-8, 0, 16777220, 16778000]
+
+        // Act
+        def results = parallelisms.collect { parallelism ->
+            def isValid = Argon2SecureHasher.isParallelismValid(parallelism)
+            [parallelism, isValid]
+        }
+
+        // Assert
+        results.each { parallelism, isParallelismValid ->
+            logger.info("For parallelization factor ${parallelism}, parallelism is ${isParallelismValid ? "valid" : "invalid"}")
+            assert !isParallelismValid
+        }
+    }
+
+    @Test
+    void testShouldVerifyIterationsBoundary() throws Exception {
+        // Arrange
+        final int iterations = 4
+
+        // Act
+        boolean valid = Argon2SecureHasher.isIterationsValid(iterations)
+
+        // Assert
+        assert valid
+    }
+
+    @Test
+    void testShouldFailIterationsBoundary() throws Exception {
+        // Arrange
+        def iterationCounts = [-50, -1, 0]
+
+        // Act
+        def results = iterationCounts.collect { iterations ->
+            def isValid = Argon2SecureHasher.isIterationsValid(iterations)
+            [iterations, isValid]
+        }
+
+        // Assert
+        results.each { iterations, isIterationsValid ->
+            logger.info("For iteration counts ${iterations}, iteration is ${isIterationsValid ? "valid" : "invalid"}")
+            assert !isIterationsValid
+        }
+    }
+
+    @Test
+    void testShouldVerifySaltLengthBoundary() throws Exception {
+        // Arrange
+        def saltLengths = [0, 64]
+
+        // Act
+        def results = saltLengths.collect { saltLength ->
+            def isValid = Argon2SecureHasher.isSaltLengthValid(saltLength)
+            [saltLength, isValid]
+        }
+
+        // Assert
+        results.each { saltLength, isSaltLengthValid ->
+            logger.info("For salt length ${saltLength}, saltLength is ${isSaltLengthValid ? "valid" : "invalid"}")
+            assert isSaltLengthValid
+        }
+    }
+
+    @Test
+    void testShouldFailSaltLengthBoundary() throws Exception {
+        // Arrange
+        def saltLengths = [-16, 4]
+
+        // Act
+        def results = saltLengths.collect { saltLength ->
+            def isValid = Argon2SecureHasher.isSaltLengthValid(saltLength)
+            [saltLength, isValid]
+        }
+
+        // Assert
+        results.each { saltLength, isSaltLengthValid ->
+            logger.info("For salt length ${saltLength}, saltLength is ${isSaltLengthValid ? "valid" : "invalid"}")
+            assert !isSaltLengthValid
+        }
+    }
 }


[nifi] 01/47: NIFI-7178 - Handle the case when schema is not available

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 1fed41641b52a451d545637a0c16692d3e6a112e
Author: Zoltan Kornel Torok <zt...@cloudera.com>
AuthorDate: Fri Feb 21 16:13:40 2020 +0100

    NIFI-7178 - Handle the case when schema is not available
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4067.
---
 .../src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java   | 4 ++--
 .../test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java   | 4 ++++
 .../src/test/resources/json/timestamp.json                            | 2 +-
 3 files changed, 7 insertions(+), 3 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
index c977cfd..74308f4 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java
@@ -180,7 +180,7 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
             return new MapRecord(childSchema, values);
         }
 
-        if (value instanceof String) {
+        if (dataType != null && value instanceof String) {
             switch (dataType.getFieldType()) {
                 case DATE:
                 case TIME:
@@ -243,4 +243,4 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
         }
     }
 
-}
\ No newline at end of file
+}
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
index d97017c..0b16c38 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonPathRowRecordReader.java
@@ -174,6 +174,7 @@ public class TestJsonPathRowRecordReader {
 
         final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>();
         jsonPaths.put("timestamp", JsonPath.compile("$.timestamp"));
+        jsonPaths.put("field_not_in_schema", JsonPath.compile("$.field_not_in_schema"));
 
         for (final boolean coerceTypes : new boolean[] {true, false}) {
             try (final InputStream in = new FileInputStream(new File("src/test/resources/json/timestamp.json"));
@@ -182,6 +183,9 @@ public class TestJsonPathRowRecordReader {
                 final Record record = reader.nextRecord(coerceTypes, false);
                 final Object value = record.getValue("timestamp");
                 assertTrue("With coerceTypes set to " + coerceTypes + ", value is not a Timestamp", value instanceof java.sql.Timestamp);
+
+                final Object valueNotInSchema = record.getValue("field_not_in_schema");
+                assertTrue("field_not_in_schema should be String", valueNotInSchema instanceof String);
             }
         }
     }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/timestamp.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/timestamp.json
index ee5d90f..2a06403 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/timestamp.json
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/timestamp.json
@@ -1 +1 @@
-{"timestamp": "2019/06/27 13:04:04"}
\ No newline at end of file
+{"timestamp": "2019/06/27 13:04:04", "field_not_in_schema" :  "some_value"}
\ No newline at end of file


[nifi] 02/47: NIFI-7185 relaxed timing constrained for builds on lower resources environments like our Github CI builds

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit d17a5aa62d71babad096e43c0353ec4d1f6e6ab9
Author: Joe Witt <jo...@apache.org>
AuthorDate: Fri Feb 21 22:24:05 2020 -0500

    NIFI-7185 relaxed timing constrained for builds on lower resources environments like our Github CI builds
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4075.
---
 .../nifi/provenance/index/lucene/TestLuceneEventIndex.java   | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
index bd4cc3d..69be483 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
@@ -154,7 +154,7 @@ public class TestLuceneEventIndex {
         List<LineageNode> nodes = Collections.emptyList();
         while (nodes.size() < 3) {
             final ComputeLineageSubmission submission = index.submitLineageComputation(1L, user, EventAuthorizer.DENY_ALL);
-            assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS));
+            assertTrue(submission.getResult().awaitCompletion(15, TimeUnit.SECONDS));
 
             nodes = submission.getResult().getNodes();
             Thread.sleep(25L);
@@ -225,7 +225,7 @@ public class TestLuceneEventIndex {
         List<LineageNode> nodes = Collections.emptyList();
         while (nodes.size() < 5) {
             final ComputeLineageSubmission submission = index.submitExpandChildren(1L, user, allowForkEvents);
-            assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS));
+            assertTrue(submission.getResult().awaitCompletion(15, TimeUnit.SECONDS));
 
             nodes = submission.getResult().getNodes();
             Thread.sleep(25L);
@@ -302,7 +302,7 @@ public class TestLuceneEventIndex {
         List<LineageNode> nodes = Collections.emptyList();
         while (nodes.size() < 2) {
             final ComputeLineageSubmission submission = index.submitExpandParents(1L, user, allowJoinEvents);
-            assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS));
+            assertTrue(submission.getResult().awaitCompletion(15, TimeUnit.SECONDS));
 
             nodes = submission.getResult().getNodes();
             Thread.sleep(25L);
@@ -354,7 +354,7 @@ public class TestLuceneEventIndex {
         List<ProvenanceEventRecord> events = Collections.emptyList();
         while (events.size() < 2) {
             final QuerySubmission submission = index.submitQuery(query, authorizer, "unit test");
-            assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS));
+            assertTrue(submission.getResult().awaitCompletion(15, TimeUnit.SECONDS));
             events = submission.getResult().getMatchingEvents();
             Thread.sleep(25L);
         }
@@ -459,7 +459,7 @@ public class TestLuceneEventIndex {
 
             final QueryResult result = submission.getResult();
             assertNotNull(result);
-            result.awaitCompletion(2000, TimeUnit.MILLISECONDS);
+            result.awaitCompletion(4000, TimeUnit.MILLISECONDS);
 
             assertTrue(result.isFinished());
             assertNull(result.getError());
@@ -502,7 +502,7 @@ public class TestLuceneEventIndex {
 
             final QueryResult result = submission.getResult();
             assertNotNull(result);
-            result.awaitCompletion(2000, TimeUnit.MILLISECONDS);
+            result.awaitCompletion(4000, TimeUnit.MILLISECONDS);
 
             assertTrue(result.isFinished());
             assertNull(result.getError());


[nifi] 46/47: NIFI-7223 - Added another test for when keyPasswd is not present.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit bdef4b4b9a2c3fade0bc925b09937be06971c3c6
Author: Nathan Gough <th...@gmail.com>
AuthorDate: Sat Mar 14 04:19:39 2020 +1100

    NIFI-7223 - Added another test for when keyPasswd is not present.
---
 .../okhttp/OkHttpReplicationClientTest.groovy      | 23 ++++++++++++++++++++++
 1 file changed, 23 insertions(+)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
index 2cfb2cb..c30c3e9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
@@ -163,6 +163,29 @@ class OkHttpReplicationClientTest extends GroovyTestCase {
     }
 
     @Test
+    void testShouldUseKeystorePasswdIfKeyPasswdIsNull() {
+        // Arrange
+        Map flowfileEncryptionProps = [
+                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+        ]
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
+
+        // Act
+        OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+
+        // Assert
+        assertNotNull(client.okHttpClient.sslSocketFactory)
+        assertEquals(SunX509KeyManagerImpl.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
+        assertNotNull(client.okHttpClient.sslSocketFactory.context.getX509KeyManager().credentialsMap["nifi-key"])    }
+
+    @Test
     void testShouldFailIfKeyPasswordIsSetButKeystorePasswordIsBlank() {
         // Arrange
         Map propsMap = [


[nifi] 12/47: NIFI-7208: Fixed PutSQL/JdbcCommon handling of timestamps (nanoseconds, e.g.)

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e1c26560055d1869188d4eb0cfb02df8e3345ce5
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Wed Feb 26 18:21:11 2020 -0500

    NIFI-7208: Fixed PutSQL/JdbcCommon handling of timestamps (nanoseconds, e.g.)
---
 .../java/org/apache/nifi/util/db/JdbcCommon.java    | 13 ++++++-------
 .../apache/nifi/processors/standard/TestPutSQL.java | 21 ++++++++++++---------
 2 files changed, 18 insertions(+), 16 deletions(-)

diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
index 34c27c2..30ac518 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
@@ -79,7 +79,6 @@ import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.time.ZoneId;
 import java.time.format.DateTimeFormatter;
-import java.time.temporal.TemporalAccessor;
 import java.util.Date;
 import java.util.Map;
 import java.util.function.Function;
@@ -773,10 +772,11 @@ public class JdbcCommon {
                     stmt.setTime(parameterIndex, time);
                     break;
                 case Types.TIMESTAMP:
-                    long lTimestamp=0L;
+                    Timestamp ts;
 
                     // Backwards compatibility note: Format was unsupported for a timestamp field.
                     if (valueFormat.equals("")) {
+                        long lTimestamp = 0L;
                         if(LONG_PATTERN.matcher(parameterValue).matches()){
                             lTimestamp = Long.parseLong(parameterValue);
                         } else {
@@ -784,15 +784,14 @@ public class JdbcCommon {
                             java.util.Date parsedDate = dateFormat.parse(parameterValue);
                             lTimestamp = parsedDate.getTime();
                         }
+                        ts = new Timestamp(lTimestamp);
                     } else {
                         final DateTimeFormatter dtFormatter = getDateTimeFormatter(valueFormat);
-                        TemporalAccessor accessor = dtFormatter.parse(parameterValue);
-                        java.util.Date parsedDate = java.util.Date.from(Instant.from(accessor));
-                        lTimestamp = parsedDate.getTime();
+                        LocalDateTime ldt = LocalDateTime.parse(parameterValue, dtFormatter);
+                        ts = Timestamp.from(ldt.atZone(ZoneId.of("UTC")).toInstant());
                     }
 
-                    stmt.setTimestamp(parameterIndex, new Timestamp(lTimestamp));
-
+                    stmt.setTimestamp(parameterIndex, ts);
                     break;
                 case Types.BINARY:
                 case Types.VARBINARY:
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java
index 5d155ee..510e384 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java
@@ -434,16 +434,20 @@ public class TestPutSQL {
         runner.enableControllerService(service);
         runner.setProperty(PutSQL.CONNECTION_POOL, "dbcp");
 
-        final String dateStr = "2002-02-02T12:02:02+00:00";
-        final long dateInt = 1012651322000L;
+        final String dateStr1 = "2002-02-02T12:02:02+00:00";
+        final long dateInt1 = 1012651322000L;
+
+        final String dateStr2 = "2002-02-02T12:02:02.123456789";
+        final long dateInt2 = 1012651322123L;
+        final long nanoInt2 = 123456789L;
 
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("sql.args.1.type", String.valueOf(Types.TIMESTAMP));
-        attributes.put("sql.args.1.value", dateStr);
+        attributes.put("sql.args.1.value", dateStr1);
         attributes.put("sql.args.1.format", "ISO_OFFSET_DATE_TIME");
         attributes.put("sql.args.2.type", String.valueOf(Types.TIMESTAMP));
-        attributes.put("sql.args.2.value", dateStr);
-        attributes.put("sql.args.2.format", "yyyy-MM-dd'T'HH:mm:ssXXX");
+        attributes.put("sql.args.2.value", dateStr2);
+        attributes.put("sql.args.2.format", "yyyy-MM-dd'T'HH:mm:ss.SSSSSSSSS");
 
         runner.enqueue("INSERT INTO TIMESTAMPTEST2 (ID, ts1, ts2) VALUES (1, ?, ?)".getBytes(), attributes);
         runner.run();
@@ -455,8 +459,9 @@ public class TestPutSQL {
                 final ResultSet rs = stmt.executeQuery("SELECT * FROM TIMESTAMPTEST2");
                 assertTrue(rs.next());
                 assertEquals(1, rs.getInt(1));
-                assertEquals(dateInt, rs.getTimestamp(2).getTime());
-                assertEquals(dateInt, rs.getTimestamp(3).getTime());
+                assertEquals(dateInt1, rs.getTimestamp(2).getTime());
+                assertEquals(dateInt2, rs.getTimestamp(3).getTime());
+                assertEquals(nanoInt2, rs.getTimestamp(3).getNanos());
                 assertFalse(rs.next());
             }
         }
@@ -477,11 +482,9 @@ public class TestPutSQL {
 
         final String dateStr = "2002-03-04";
         final String timeStr = "02:03:04";
-
         final String timeFormatString = "HH:mm:ss";
         final String dateFormatString ="yyyy-MM-dd";
 
-
         final DateTimeFormatter timeFormatter= DateTimeFormatter.ISO_LOCAL_TIME;
         LocalTime parsedTime = LocalTime.parse(timeStr, timeFormatter);
         Time expectedTime = Time.valueOf(parsedTime);


[nifi] 36/47: NIFI-7244 Updated all tests which dont run well on windows to either work or be ignored on windows Also dealt with unreliable tests which depend on timing by ignoring them or converting to IT.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0d5cd830b7886ae4ca0b3bfa559cd924365ff04d
Author: Joe Witt <jo...@apache.org>
AuthorDate: Tue Mar 10 15:14:01 2020 -0400

    NIFI-7244 Updated all tests which dont run well on windows to either work or be ignored on windows
    Also dealt with unreliable tests which depend on timing by ignoring them or converting to IT.
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4132.
---
 .github/workflows/ci-workflow.yml                    | 16 ++++++++++------
 nifi-commons/nifi-rocksdb-utils/pom.xml              |  6 ++++++
 .../apache/nifi/rocksdb/TestRocksDBMetronome.java    | 14 ++++++++------
 .../apache/nifi/accumulo/processors/PutRecordIT.java |  3 +++
 .../nifi/accumulo/processors/ScanAccumuloIT.java     |  3 +++
 .../nifi/authorization/FileAuthorizerTest.java       |  8 ++++++++
 .../EncryptedRepositoryRecordSerdeFactoryTest.groovy |  4 +++-
 .../EncryptedSchemaRepositoryRecordSerdeTest.groovy  |  3 +++
 .../protocol/impl/SocketProtocolListenerTest.java    |  2 ++
 .../heartbeat/TestAbstractHeartbeatMonitor.java      |  2 ++
 .../crypto/EncryptedFileSystemRepositoryTest.groovy  |  3 +++
 ...EncryptedSequentialAccessWriteAheadLogTest.groovy |  3 +++
 .../repository/TestFileSystemRepository.java         | 11 ++++++++---
 .../repository/TestRocksDBFlowFileRepository.java    | 14 ++++++++------
 .../nifi-framework-nar-loading-utils/pom.xml         |  6 ++++++
 .../org/apache/nifi/nar/AbstractTestNarLoader.java   |  8 ++++++++
 .../test/java/org/apache/nifi/nar/TestNarLoader.java |  8 ++++++++
 .../AESSensitivePropertyProviderFactoryTest.groovy   |  3 +++
 .../properties/NiFiPropertiesLoaderGroovyTest.groovy |  3 +++
 .../ProtectedNiFiPropertiesGroovyTest.groovy         |  3 +++
 .../nifi/web/security/knox/KnoxServiceTest.java      |  8 ++++++++
 .../processors/groovyx/ExecuteGroovyScriptTest.java  |  3 +++
 .../apache/nifi/processors/hadoop/GetHDFSTest.java   |  8 ++++++++
 .../apache/nifi/processors/hadoop/MoveHDFSTest.java  |  8 ++++++++
 .../nifi/processors/hive/TestPutHive3Streaming.java  |  8 ++++++++
 .../org/apache/nifi/processors/orc/PutORCTest.java   |  5 ++++-
 .../jolt/record/TestJoltTransformRecord.java         |  9 +++++++++
 .../nifi-kite-bundle/nifi-kite-processors/pom.xml    |  2 +-
 .../nifi/processors/kite/TestCSVToAvroProcessor.java |  7 +++++++
 .../processors/kite/TestConfigurationProperty.java   |  8 ++++++++
 .../nifi/processors/kite/TestInferAvroSchema.java    |  8 ++++++++
 .../org/apache/nifi/parquet/TestParquetReader.java   |  8 ++++++++
 .../nifi/processors/parquet/FetchParquetTest.java    |  8 ++++++++
 .../nifi/processors/parquet/PutParquetTest.java      |  5 ++++-
 .../poi/ConvertExcelToCSVProcessorTest.java          |  8 ++++++++
 .../index/lucene/TestLuceneEventIndex.java           |  2 ++
 .../TestSiteToSiteMetricsReportingTask.java          |  8 ++++++++
 .../nifi/processors/standard/TestConvertRecord.java  |  9 +++++++++
 .../nifi/processors/standard/TestEncryptContent.java |  7 +++++++
 .../standard/TestExecuteStreamCommand.java           |  3 +++
 .../nifi/processors/standard/TestFetchFile.java      |  8 ++++++++
 .../nifi/processors/standard/TestForkRecord.java     |  9 +++++++++
 .../nifi/processors/standard/TestInvokeHTTP.java     |  7 ++++++-
 .../nifi/processors/standard/TestInvokeHttpSSL.java  |  8 +++++++-
 .../processors/standard/TestInvokeHttpTwoWaySSL.java |  3 +++
 .../nifi/processors/standard/TestListFile.java       |  8 ++++++++
 .../nifi/processors/standard/TestListenHTTP.java     | 10 +++++++++-
 .../apache/nifi/processors/standard/TestPutFile.java |  8 ++++++++
 .../nifi/processors/standard/TestUpdateRecord.java   |  9 +++++++++
 .../standard/util/TestInvokeHttpCommon.java          |  1 -
 .../processors/standard/util/TestPutTCPCommon.java   |  8 ++++++++
 .../util/crypto/OpenPGPKeyBasedEncryptorTest.java    |  3 +++
 .../crypto/OpenPGPPasswordBasedEncryptorTest.java    |  3 +++
 .../org/apache/nifi/json/TestWriteJsonResult.java    |  5 +++--
 ...lingWindow.java => AttributeRollingWindowIT.java} |  2 +-
 .../admin/configmigrator/ConfigMigratorSpec.groovy   |  7 ++++++-
 .../admin/filemanager/FileManagerToolSpec.groovy     |  6 ++++++
 .../apache/nifi/toolkit/cli/TestCLICompleter.java    |  3 +++
 .../toolkit/cli/impl/result/TestBucketsResult.java   |  8 ++++++++
 .../cli/impl/result/TestRegistryClientResult.java    |  8 ++++++++
 .../TestVersionedFlowSnapshotMetadataResult.java     |  8 ++++++++
 .../cli/impl/result/TestVersionedFlowsResult.java    |  8 ++++++++
 .../impl/result/writer/TestDynamicTableWriter.java   |  8 ++++++++
 .../nifi/properties/ConfigEncryptionToolTest.groovy  | 20 ++++++++++++--------
 nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml      |  6 ++++++
 .../flowfile/TestRepairCorruptedFileEndings.java     |  8 ++++++++
 66 files changed, 396 insertions(+), 41 deletions(-)

diff --git a/.github/workflows/ci-workflow.yml b/.github/workflows/ci-workflow.yml
index 492663d..d8800fb 100644
--- a/.github/workflows/ci-workflow.yml
+++ b/.github/workflows/ci-workflow.yml
@@ -7,7 +7,7 @@ jobs:
 
     timeout-minutes: 90
     runs-on: ubuntu-latest
-    name: Ubuntu Build NiFi - JDK 1.8 FR
+    name: Ubuntu - JDK 1.8 FR
 
     steps:
     - name: Checkout Code
@@ -63,7 +63,7 @@ jobs:
 
     timeout-minutes: 90
     runs-on: ubuntu-latest
-    name: Ubuntu Build NiFi - JDK 11 EN
+    name: Ubuntu - JDK 11 EN
 
     steps:
       - name: Checkout Code
@@ -119,7 +119,7 @@ jobs:
 
     timeout-minutes: 90
     runs-on: macos-latest
-    name: MacOS Build NiFi - JDK 1.8 JP
+    name: MacOS - JDK 1.8 JP
 
     steps:
       - name: Checkout Code
@@ -174,9 +174,13 @@ jobs:
   windows-build:
 
     runs-on: windows-latest
-    name: Windows Build NiFi - JDK 1.8
+    name: Windows - JDK 1.8
 
     steps:
+      - name: Setup Git
+        run:  |
+          git config --global core.autocrlf false
+          git config --global core.longpaths true
       - name: Checkout Code
         uses: actions/checkout@v2
       - name: Check Maven Com Cache
@@ -216,5 +220,5 @@ jobs:
           MAVEN_OPTS: -Xmx2g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN
         run: |
           mvn -version
-          mvn -T 1C install -B -Ddir-only -ntp -ff -DskipTests
-          Remove-Item -path '~\.m2\repository\org\apache\nifi' -recurse -force
\ No newline at end of file
+          mvn -T 1C install -B -Ddir-only -ntp -ff
+          Remove-Item -path '~\.m2\repository\org\apache\nifi' -recurse -force
diff --git a/nifi-commons/nifi-rocksdb-utils/pom.xml b/nifi-commons/nifi-rocksdb-utils/pom.xml
index 759cf0c..7cde363 100644
--- a/nifi-commons/nifi-rocksdb-utils/pom.xml
+++ b/nifi-commons/nifi-rocksdb-utils/pom.xml
@@ -39,5 +39,11 @@
             <artifactId>rocksdbjni</artifactId>
             <version>6.2.2</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.9</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-commons/nifi-rocksdb-utils/src/test/java/org/apache/nifi/rocksdb/TestRocksDBMetronome.java b/nifi-commons/nifi-rocksdb-utils/src/test/java/org/apache/nifi/rocksdb/TestRocksDBMetronome.java
index 7b5e68b..402f099 100644
--- a/nifi-commons/nifi-rocksdb-utils/src/test/java/org/apache/nifi/rocksdb/TestRocksDBMetronome.java
+++ b/nifi-commons/nifi-rocksdb-utils/src/test/java/org/apache/nifi/rocksdb/TestRocksDBMetronome.java
@@ -16,8 +16,11 @@
  */
 package org.apache.nifi.rocksdb;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.junit.After;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -53,6 +56,11 @@ public class TestRocksDBMetronome {
 
     private ExecutorService executor;
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void before() {
         executor = Executors.newSingleThreadExecutor();
@@ -92,7 +100,6 @@ public class TestRocksDBMetronome {
 
     @Test
     public void testPutGetDelete() throws Exception {
-
         try (RocksDBMetronome db = new RocksDBMetronome.Builder()
                 .setStoragePath(temporaryFolder.newFolder().toPath())
                 .build()) {
@@ -116,7 +123,6 @@ public class TestRocksDBMetronome {
 
     @Test
     public void testPutGetConfiguration() throws Exception {
-
         try (RocksDBMetronome db = new RocksDBMetronome.Builder()
                 .setStoragePath(temporaryFolder.newFolder().toPath())
                 .build()) {
@@ -132,7 +138,6 @@ public class TestRocksDBMetronome {
 
     @Test(expected = IllegalStateException.class)
     public void testPutBeforeInit() throws Exception {
-
         try (RocksDBMetronome db = new RocksDBMetronome.Builder()
                 .setStoragePath(temporaryFolder.newFolder().toPath())
                 .build()) {
@@ -142,7 +147,6 @@ public class TestRocksDBMetronome {
 
     @Test(expected = IllegalStateException.class)
     public void testPutClosed() throws Exception {
-
         try (RocksDBMetronome db = new RocksDBMetronome.Builder()
                 .setStoragePath(temporaryFolder.newFolder().toPath())
                 .build()) {
@@ -155,7 +159,6 @@ public class TestRocksDBMetronome {
 
     @Test
     public void testColumnFamilies() throws Exception {
-
         String secondFamilyName = "second family";
         try (RocksDBMetronome db = new RocksDBMetronome.Builder()
                 .setStoragePath(temporaryFolder.newFolder().toPath())
@@ -208,7 +211,6 @@ public class TestRocksDBMetronome {
 
     @Test
     public void testIterator() throws Exception {
-
         try (RocksDBMetronome db = new RocksDBMetronome.Builder()
                 .setStoragePath(temporaryFolder.newFolder().toPath())
                 .build()) {
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/PutRecordIT.java b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/PutRecordIT.java
index c8bf47c..eaba0b8 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/PutRecordIT.java
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/PutRecordIT.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.serialization.record.MockRecordParser;
@@ -36,6 +37,7 @@ import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.nifi.accumulo.controllerservices.AccumuloService;
@@ -74,6 +76,7 @@ public class PutRecordIT {
 
     @BeforeClass
     public static void setupInstance() throws IOException, InterruptedException, AccumuloSecurityException, AccumuloException, TableExistsException {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         Path tempDirectory = Files.createTempDirectory("acc"); // JUnit and Guava supply mechanisms for creating temp directories
         accumulo = new MiniAccumuloCluster(tempDirectory.toFile(), "password");
         accumulo.start();
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/ScanAccumuloIT.java b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/ScanAccumuloIT.java
index 4abd8cb..2eaefef 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/ScanAccumuloIT.java
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/ScanAccumuloIT.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.nifi.accumulo.controllerservices.AccumuloService;
 import org.apache.nifi.accumulo.controllerservices.MockAccumuloService;
@@ -40,6 +41,7 @@ import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -75,6 +77,7 @@ public class ScanAccumuloIT {
 
     @BeforeClass
     public static void setupInstance() throws IOException, InterruptedException, AccumuloSecurityException, AccumuloException, TableExistsException {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         Path tempDirectory = Files.createTempDirectory("acc"); // JUnit and Guava supply mechanisms for creating temp directories
         accumulo = new MiniAccumuloCluster(tempDirectory.toFile(), "password");
         accumulo.start();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
index e83e97a..1b8918e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
@@ -26,7 +27,9 @@ import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.file.FileUtils;
 import org.junit.After;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
@@ -158,6 +161,11 @@ public class FileAuthorizerTest {
 
     private AuthorizerConfigurationContext configurationContext;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws IOException {
         // primary authorizations
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactoryTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactoryTest.groovy
index 5a504dd..f12a695 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactoryTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactoryTest.groovy
@@ -17,7 +17,7 @@
 
 package org.apache.nifi.controller.repository
 
-
+import org.apache.commons.lang3.SystemUtils
 import org.apache.nifi.controller.repository.claim.ResourceClaimManager
 import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager
 import org.apache.nifi.properties.StandardNiFiProperties
@@ -25,6 +25,7 @@ import org.apache.nifi.security.kms.EncryptionException
 import org.apache.nifi.util.NiFiProperties
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Rule
@@ -56,6 +57,7 @@ class EncryptedRepositoryRecordSerdeFactoryTest extends GroovyTestCase {
 
     @BeforeClass
     static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerdeTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerdeTest.groovy
index fcf3c98..9e0cf32 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerdeTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/groovy/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerdeTest.groovy
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.controller.repository
 
+import org.apache.commons.lang3.SystemUtils
 import org.apache.nifi.controller.queue.FlowFileQueue
 import org.apache.nifi.controller.repository.claim.ResourceClaimManager
 import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager
@@ -25,6 +26,7 @@ import org.apache.nifi.security.repository.config.FlowFileRepositoryEncryptionCo
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.bouncycastle.util.encoders.Hex
 import org.junit.After
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Rule
@@ -72,6 +74,7 @@ class EncryptedSchemaRepositoryRecordSerdeTest extends GroovyTestCase {
 
     @BeforeClass
     static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java
index cac5bf2..128399c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java
@@ -36,6 +36,7 @@ import org.junit.After;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 /**
@@ -109,6 +110,7 @@ public class SocketProtocolListenerTest {
         assertEquals(msg.getType(), handler.getMessages().get(0).getType());
     }
 
+    @Ignore("this test is unreliable on slow build environments")
     @Test
     public void testDelayedRequest() throws Exception {
         ProtocolMessage msg = new PingMessage();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
index 2245c6e..0a2a509 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
@@ -32,6 +32,7 @@ import org.apache.nifi.services.FlowService;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -135,6 +136,7 @@ public class TestAbstractHeartbeatMonitor {
         assertTrue(requestedToConnect.isEmpty());
     }
 
+    @Ignore("this test is too unstable in terms of timing on different size/types of testing envs")
     @Test
     public void testDisconnectionOfTerminatedNodeDueToLackOfHeartbeat() throws Exception {
         final NodeIdentifier nodeId1 = nodeId;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/repository/crypto/EncryptedFileSystemRepositoryTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/repository/crypto/EncryptedFileSystemRepositoryTest.groovy
index 4dbd7c1..0b5ce89 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/repository/crypto/EncryptedFileSystemRepositoryTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/repository/crypto/EncryptedFileSystemRepositoryTest.groovy
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.controller.repository.crypto
 
+import org.apache.commons.lang3.SystemUtils
 import org.apache.nifi.controller.repository.claim.ContentClaim
 import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager
 import org.apache.nifi.controller.repository.util.DiskUtils
@@ -30,6 +31,7 @@ import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.bouncycastle.util.encoders.Hex
 import org.junit.After
 import org.junit.AfterClass
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Test
@@ -93,6 +95,7 @@ class EncryptedFileSystemRepositoryTest {
 
     @BeforeClass
     static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         ORIGINAL_LOG_LEVEL = System.getProperty(LOG_PACKAGE)
         System.setProperty(LOG_PACKAGE, "DEBUG")
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy
index 3c978f1..d653eee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy
@@ -18,6 +18,7 @@
 package org.apache.nifi.wali
 
 import ch.qos.logback.classic.Level
+import org.apache.commons.lang3.SystemUtils
 import org.apache.nifi.controller.queue.FlowFileQueue
 import org.apache.nifi.controller.repository.EncryptedSchemaRepositoryRecordSerde
 import org.apache.nifi.controller.repository.RepositoryRecord
@@ -32,6 +33,7 @@ import org.apache.nifi.security.kms.CryptoUtils
 import org.apache.nifi.security.repository.config.FlowFileRepositoryEncryptionConfiguration
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Rule
@@ -88,6 +90,7 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
 
     @BeforeClass
     static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
index 6ac10d5..5657a11 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
@@ -20,6 +20,7 @@ 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 org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.controller.repository.claim.ContentClaim;
 import org.apache.nifi.controller.repository.claim.ResourceClaim;
 import org.apache.nifi.controller.repository.claim.StandardContentClaim;
@@ -31,7 +32,9 @@ import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.LoggerFactory;
@@ -66,7 +69,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeFalse;
 
 public class TestFileSystemRepository {
 
@@ -79,6 +81,11 @@ public class TestFileSystemRepository {
     private final File rootFile = new File("target/content_repository");
     private NiFiProperties nifiProperties;
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws IOException {
         System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile());
@@ -518,7 +525,6 @@ public class TestFileSystemRepository {
 
     @Test
     public void testReadWithContentArchived() throws IOException {
-        assumeFalse(isWindowsEnvironment());//skip if on windows
         final ContentClaim claim = repository.create(true);
         final Path path = getPath(claim);
         Files.deleteIfExists(path);
@@ -544,7 +550,6 @@ public class TestFileSystemRepository {
 
     @Test(expected = ContentNotFoundException.class)
     public void testReadWithNoContentArchived() throws IOException {
-        assumeFalse(isWindowsEnvironment());//skip if on windows
         final ContentClaim claim = repository.create(true);
         final Path path = getPath(claim);
         Files.deleteIfExists(path);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestRocksDBFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestRocksDBFlowFileRepository.java
index 3bf38ca..cebf2cd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestRocksDBFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestRocksDBFlowFileRepository.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.controller.repository;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.queue.FlowFileQueue;
@@ -32,7 +33,9 @@ import org.apache.nifi.controller.swap.StandardSwapSummary;
 import org.apache.nifi.rocksdb.RocksDBMetronome;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.file.FileUtils;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -84,6 +87,11 @@ public class TestRocksDBFlowFileRepository {
     @Rule
     public TemporaryFolder temporaryFolder = new TemporaryFolder();
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void before() throws IOException {
         File testRepoDir = temporaryFolder.newFolder(testName.getMethodName());
@@ -113,7 +121,6 @@ public class TestRocksDBFlowFileRepository {
 
     @Test
     public void testSwapLocationsRestored() throws IOException {
-
         final RocksDBFlowFileRepository repo = new RocksDBFlowFileRepository(NiFiProperties.createBasicNiFiProperties(nifiPropertiesPath, additionalProperties));
         repo.initialize(new StandardResourceClaimManager());
 
@@ -274,7 +281,6 @@ public class TestRocksDBFlowFileRepository {
 
     @Test
     public void testRestartWithOneRecord() throws IOException {
-
         final RocksDBFlowFileRepository repo = new RocksDBFlowFileRepository(NiFiProperties.createBasicNiFiProperties(nifiPropertiesPath, additionalProperties));
         repo.initialize(new StandardResourceClaimManager());
 
@@ -343,7 +349,6 @@ public class TestRocksDBFlowFileRepository {
 
     @Test
     public void testDoNotRemoveOrphans() throws Exception {
-
         final TestQueue testQueue = new TestQueue();
 
         try (final RocksDBFlowFileRepository repo = new RocksDBFlowFileRepository(NiFiProperties.createBasicNiFiProperties(nifiPropertiesPath, additionalProperties))) {
@@ -378,7 +383,6 @@ public class TestRocksDBFlowFileRepository {
 
     @Test
     public void testRemoveOrphans() throws Exception {
-
         final TestQueue testQueue = new TestQueue();
 
         additionalProperties.put(RocksDBFlowFileRepository.RocksDbProperty.REMOVE_ORPHANED_FLOWFILES.propertyName, "true");
@@ -440,7 +444,6 @@ public class TestRocksDBFlowFileRepository {
 
     @Test
     public void testRecoveryMode() throws Exception {
-
         int totalFlowFiles = 50;
 
         final TestQueue testQueue = new TestQueue();
@@ -503,7 +506,6 @@ public class TestRocksDBFlowFileRepository {
 
     @Test
     public void testRecoveryModeWithContinuedLoading() throws Exception {
-
         additionalProperties.put(RocksDBFlowFileRepository.RocksDbProperty.CLAIM_CLEANUP_PERIOD.propertyName, "24 hours"); // "disable" the cleanup thread, let us manually force recovery
 
         int totalFlowFiles = 50;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
index 67ad0bf..90dcdb2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
@@ -42,5 +42,11 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.9</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java
index 70daa63..22fd133 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java
@@ -16,12 +16,15 @@
  */
 package org.apache.nifi.nar;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.util.NiFiProperties;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 
 import java.io.File;
 import java.io.IOException;
@@ -49,6 +52,11 @@ public abstract class AbstractTestNarLoader {
     NarClassLoaders narClassLoaders;
     ExtensionDiscoveringManager extensionManager;
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws IOException, ClassNotFoundException {
         deleteDir(getWorkDir());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
index 8fdb4b3..94a08d6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
@@ -16,9 +16,12 @@
  */
 package org.apache.nifi.nar;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.reporting.ReportingTask;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.File;
@@ -39,6 +42,11 @@ public class TestNarLoader extends AbstractTestNarLoader {
     static final String PROPERTIES_FILE = "./src/test/resources/conf/nifi.properties";
     static final String EXTENSIONS_DIR = "./src/test/resources/extensions";
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Test
     public void testNarLoaderWhenAllAvailable() throws IOException {
         // Copy all NARs from src/test/resources/extensions to target/extensions
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderFactoryTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderFactoryTest.groovy
index 115fca6..26352c0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderFactoryTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderFactoryTest.groovy
@@ -16,8 +16,10 @@
  */
 package org.apache.nifi.properties
 
+import org.apache.commons.lang3.SystemUtils
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Ignore
@@ -37,6 +39,7 @@ class AESSensitivePropertyProviderFactoryTest extends GroovyTestCase {
 
     @BeforeClass
     public static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/NiFiPropertiesLoaderGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/NiFiPropertiesLoaderGroovyTest.groovy
index 53f4070..9538dc8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/NiFiPropertiesLoaderGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/NiFiPropertiesLoaderGroovyTest.groovy
@@ -16,10 +16,12 @@
  */
 package org.apache.nifi.properties
 
+import org.apache.commons.lang3.SystemUtils
 import org.apache.nifi.util.NiFiProperties
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After
 import org.junit.AfterClass
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Ignore
@@ -68,6 +70,7 @@ class NiFiPropertiesLoaderGroovyTest extends GroovyTestCase {
 
     @BeforeClass
     static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
index 6be470f..034c6ca 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
@@ -16,10 +16,12 @@
  */
 package org.apache.nifi.properties
 
+import org.apache.commons.lang3.SystemUtils
 import org.apache.nifi.util.NiFiProperties
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After
 import org.junit.AfterClass
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Test
@@ -55,6 +57,7 @@ class ProtectedNiFiPropertiesGroovyTest extends GroovyTestCase {
 
     @BeforeClass
     static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java
index 709a303..3940b13 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/knox/KnoxServiceTest.java
@@ -24,8 +24,11 @@ import com.nimbusds.oauth2.sdk.auth.PrivateKeyJWT;
 import com.nimbusds.oauth2.sdk.id.Audience;
 import com.nimbusds.oauth2.sdk.id.ClientID;
 import com.nimbusds.oauth2.sdk.id.JWTID;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.web.security.InvalidAuthenticationException;
 import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.security.KeyPair;
@@ -47,6 +50,11 @@ public class KnoxServiceTest {
     private static final String AUDIENCE = "https://apache-knox/token";
     private static final String AUDIENCE_2 = "https://apache-knox-2/token";
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Test(expected = IllegalStateException.class)
     public void testKnoxSsoNotEnabledGetKnoxUrl() throws Exception {
         final KnoxConfiguration configuration = mock(KnoxConfiguration.class);
diff --git a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/test/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScriptTest.java b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/test/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScriptTest.java
index 81d1857..473a480 100644
--- a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/test/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScriptTest.java
+++ b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/test/java/org/apache/nifi/processors/groovyx/ExecuteGroovyScriptTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.groovyx;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.MockRecordParser;
@@ -34,6 +35,7 @@ import org.apache.nifi.util.TestRunners;
 import org.apache.nifi.processor.exception.ProcessException;
 
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Before;
 import org.junit.Test;
@@ -100,6 +102,7 @@ public class ExecuteGroovyScriptTest {
      */
     @BeforeClass
     public static void setupBeforeClass() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         FileUtils.copyDirectory(new File("src/test/resources"), new File("target/test/resources"));
         //prepare database connection
         System.setProperty("derby.stream.error.file", "target/derby.log");
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java
index d3837a8..fcba96e 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/GetHDFSTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.hadoop;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -29,7 +30,9 @@ import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.File;
@@ -49,6 +52,11 @@ public class GetHDFSTest {
     private NiFiProperties mockNiFiProperties;
     private KerberosProperties kerberosProperties;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         mockNiFiProperties = mock(NiFiProperties.class);
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/MoveHDFSTest.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/MoveHDFSTest.java
index 7d14b5d..a256f6e 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/MoveHDFSTest.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/MoveHDFSTest.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.hadoop;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.hadoop.KerberosProperties;
 import org.apache.nifi.processor.ProcessContext;
@@ -28,7 +29,9 @@ import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.File;
@@ -52,6 +55,11 @@ public class MoveHDFSTest {
     private NiFiProperties mockNiFiProperties;
     private KerberosProperties kerberosProperties;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         mockNiFiProperties = mock(NiFiProperties.class);
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
index 16c42e5..a572b45 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
@@ -26,6 +26,7 @@ import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.io.DatumWriter;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -76,7 +77,9 @@ import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.apache.nifi.util.hive.HiveConfigurator;
 import org.apache.nifi.util.hive.HiveOptions;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.ByteArrayInputStream;
@@ -136,6 +139,11 @@ public class TestPutHive3Streaming {
     private UserGroupInformation ugi;
     private Schema schema;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setUp() throws Exception {
 
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
index c063ea1..2df3467 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.orc;
 
 import org.apache.avro.Schema;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -55,6 +56,7 @@ import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -95,7 +97,8 @@ public class PutORCTest {
     private TestRunner testRunner;
 
     @BeforeClass
-    public static void setupLogging() {
+    public static void setupBeforeClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         BasicConfigurator.configure();
     }
 
diff --git a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java
index 1c5365f..3e28b0d 100644
--- a/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java
+++ b/nifi-nar-bundles/nifi-jolt-record-bundle/nifi-jolt-record-processors/src/test/java/org/apache/nifi/processors/jolt/record/TestJoltTransformRecord.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.jolt.record;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.json.JsonRecordSetWriter;
 import org.apache.nifi.processor.Relationship;
@@ -32,7 +33,9 @@ import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.StringUtils;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -55,6 +58,12 @@ public class TestJoltTransformRecord {
     private MockRecordParser parser;
     private JsonRecordSetWriter writer;
 
+    //The pretty printed json comparisons dont work on windows
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws Exception {
         processor = new JoltTransformRecord();
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml
index 65051b2..4ec9699 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml
@@ -60,7 +60,7 @@
         <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-lang3</artifactId>
-            <version>3.8.1</version>
+            <version>3.9</version>
         </dependency>
 
         <dependency>
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestCSVToAvroProcessor.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestCSVToAvroProcessor.java
index 50f5599..58d9ce8 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestCSVToAvroProcessor.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestCSVToAvroProcessor.java
@@ -32,11 +32,14 @@ import org.apache.avro.file.DataFileStream;
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.io.DatumReader;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.processors.kite.AbstractKiteConvertProcessor.CodecType;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.apache.nifi.processors.kite.TestUtil.streamFor;
@@ -68,6 +71,10 @@ public class TestCSVToAvroProcessor {
     public static final String FAILURE_SUMMARY = "" +
             "Field id: cannot make \"long\" value: '': Field id type:LONG pos:0 not set and has no default value";
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
 
     /**
      * Test for a schema that is not a JSON but does not throw exception when trying to parse as an URI
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConfigurationProperty.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConfigurationProperty.java
index ef55710..a8d9d75 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConfigurationProperty.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConfigurationProperty.java
@@ -22,12 +22,15 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import org.apache.avro.generic.GenericData.Record;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.After;
@@ -46,6 +49,11 @@ public class TestConfigurationProperty {
     private String datasetUri = null;
     private Dataset<Record> dataset = null;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void saveConfiguration() throws IOException {
         Configuration conf = new Configuration(false);
diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
index e1a9ef9..d76214a 100644
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
@@ -18,6 +18,7 @@
  */
 package org.apache.nifi.processors.kite;
 
+import org.apache.commons.lang3.SystemUtils;
 import java.io.BufferedInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -40,9 +41,16 @@ import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 
 public class TestInferAvroSchema {
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     private TestRunner runner = null;
 
     @Before
diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java
index da54242..1dcf721 100644
--- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java
+++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/parquet/TestParquetReader.java
@@ -20,6 +20,7 @@ import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -31,7 +32,9 @@ import org.apache.nifi.util.MockComponentLog;
 import org.apache.nifi.util.MockConfigurationContext;
 import org.apache.parquet.avro.AvroParquetWriter;
 import org.apache.parquet.hadoop.ParquetWriter;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.ByteArrayInputStream;
@@ -54,6 +57,11 @@ public class TestParquetReader {
     private ParquetReader parquetReaderFactory;
     private ComponentLog componentLog;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         readerFactoryProperties = new HashMap<>();
diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/FetchParquetTest.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/FetchParquetTest.java
index 58fbd3e..ccb9d47 100644
--- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/FetchParquetTest.java
+++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/FetchParquetTest.java
@@ -21,6 +21,7 @@ import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.nifi.flowfile.FlowFile;
@@ -41,7 +42,9 @@ import org.apache.nifi.util.TestRunners;
 import org.apache.parquet.avro.AvroParquetWriter;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.AdditionalMatchers;
 import org.mockito.Mockito;
@@ -75,6 +78,11 @@ public class FetchParquetTest {
     private FetchParquet proc;
     private TestRunner testRunner;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws IOException, InitializationException {
         final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/avro/user.avsc"), StandardCharsets.UTF_8);
diff --git a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java
index c1bcbc7..b4ba3db 100644
--- a/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java
+++ b/nifi-nar-bundles/nifi-parquet-bundle/nifi-parquet-processors/src/test/java/org/apache/nifi/processors/parquet/PutParquetTest.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -61,6 +62,7 @@ import org.apache.parquet.avro.AvroParquetReader;
 import org.apache.parquet.hadoop.ParquetReader;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -79,7 +81,8 @@ public class PutParquetTest {
     private TestRunner testRunner;
 
     @BeforeClass
-    public static void setupLogging() {
+    public static void setupBeforeClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         BasicConfigurator.configure();
     }
 
diff --git a/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/test/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessorTest.java b/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/test/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessorTest.java
index 828e94c..8a36da8 100644
--- a/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/test/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessorTest.java
+++ b/nifi-nar-bundles/nifi-poi-bundle/nifi-poi-processors/src/test/java/org/apache/nifi/processors/poi/ConvertExcelToCSVProcessorTest.java
@@ -27,6 +27,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.csv.CSVUtils;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.util.LogMessage;
@@ -36,11 +37,18 @@ import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 
 public class ConvertExcelToCSVProcessorTest {
 
     private TestRunner testRunner;
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void init() {
         testRunner = TestRunners.newTestRunner(ConvertExcelToCSVProcessor.class);
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
index d3733bd..959f71b 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
@@ -42,6 +42,7 @@ import org.apache.nifi.provenance.store.StorageResult;
 import org.apache.nifi.util.Tuple;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -175,6 +176,7 @@ public class TestLuceneEventIndex {
         }
     }
 
+    @Ignore("This test is unreliable in certain build environments")
     @Test(timeout = 60000)
     public void testUnauthorizedEventsGetPlaceholdersForExpandChildren() throws InterruptedException, IOException {
         assumeFalse(isWindowsEnvironment());
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteMetricsReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteMetricsReportingTask.java
index 3b81c26..6db84ef 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteMetricsReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteMetricsReportingTask.java
@@ -38,6 +38,7 @@ import javax.json.JsonObject;
 import javax.json.JsonReader;
 import javax.json.JsonValue;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
@@ -55,7 +56,9 @@ import org.apache.nifi.serialization.record.MockRecordWriter;
 import org.apache.nifi.state.MockStateManager;
 import org.apache.nifi.util.MockPropertyValue;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
@@ -66,6 +69,11 @@ public class TestSiteToSiteMetricsReportingTask {
     private ReportingContext context;
     private ProcessGroupStatus status;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         status = new ProcessGroupStatus();
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java
index 822f664..1d0cde1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java
@@ -30,6 +30,7 @@ import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.csv.CSVReader;
 import org.apache.nifi.csv.CSVRecordSetWriter;
 import org.apache.nifi.csv.CSVUtils;
@@ -43,11 +44,19 @@ import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.xerial.snappy.SnappyInputStream;
 
 public class TestConvertRecord {
 
+    //Apparently pretty printing is not portable as these tests fail on windows
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Test
     public void testSuccessfulConversion() throws InitializationException {
         final MockRecordParser readerService = new MockRecordParser();
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java
index 6464665..4dd0359 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEncryptContent.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Objects;
 
 import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.security.util.EncryptionMethod;
@@ -45,6 +46,7 @@ import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -68,6 +70,11 @@ public class TestEncryptContent {
         return null;
     }
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setUp() {
         Security.addProvider(new BouncyCastleProvider());
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
index c8b8763..f36ded4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java
@@ -36,12 +36,14 @@ import java.util.regex.Pattern;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processors.standard.util.ArgumentUtils;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -49,6 +51,7 @@ import org.junit.Test;
 public class TestExecuteStreamCommand {
     @BeforeClass
     public static void init() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
         System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
         System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.ExecuteStreamCommand", "debug");
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFile.java
index ee2367c..c50b4f6 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFile.java
@@ -27,13 +27,21 @@ import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
 import java.util.Arrays;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestFetchFile {
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void prepDestDirectory() throws IOException {
         final File targetDir = new File("target/move-target");
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java
index 70ec39f..05bace4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestForkRecord.java
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.processors.standard;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.json.JsonRecordSetWriter;
@@ -38,6 +39,8 @@ import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.File;
@@ -93,6 +96,12 @@ public class TestForkRecord {
         return new SimpleRecordSchema(transactionFields);
     }
 
+    //Pretty printing is not portable as these fail on windows
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Test
     public void testForkExtractSimpleWithoutParentFields() throws IOException, MalformedRecordException, InitializationException {
         TestRunner runner = TestRunners.newTestRunner(new ForkRecord());
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
index e25344c..02609c0 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.standard;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.processors.standard.util.TestInvokeHttpCommon;
 import org.apache.nifi.web.util.TestServer;
 import org.apache.nifi.ssl.StandardSSLContextService;
@@ -26,6 +27,7 @@ import org.eclipse.jetty.server.handler.AbstractHandler;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -48,6 +50,7 @@ public class TestInvokeHTTP extends TestInvokeHttpCommon {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         // useful for verbose logging output
         // don't commit this with this property enabled, or any 'mvn test' will be really verbose
         // System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard", "debug");
@@ -62,7 +65,9 @@ public class TestInvokeHTTP extends TestInvokeHttpCommon {
 
     @AfterClass
     public static void afterClass() throws Exception {
-        server.shutdownServer();
+        if(server != null) {
+            server.shutdownServer();
+        }
     }
 
     @Before
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
index 0dce0f0..4fbc562 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
@@ -20,12 +20,15 @@ package org.apache.nifi.processors.standard;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
+
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.processors.standard.util.TestInvokeHttpCommon;
 import org.apache.nifi.ssl.StandardSSLContextService;
 import org.apache.nifi.util.TestRunners;
 import org.apache.nifi.web.util.TestServer;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
 
@@ -41,6 +44,7 @@ public class TestInvokeHttpSSL extends TestInvokeHttpCommon {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         // useful for verbose logging output
         // don't commit this with this property enabled, or any 'mvn test' will be really verbose
         // System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard", "debug");
@@ -62,7 +66,9 @@ public class TestInvokeHttpSSL extends TestInvokeHttpCommon {
 
     @AfterClass
     public static void afterClass() throws Exception {
-        server.shutdownServer();
+        if(server != null) {
+            server.shutdownServer();
+        }
     }
 
     @Before
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpTwoWaySSL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpTwoWaySSL.java
index 3ca4cd3..4e500b9 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpTwoWaySSL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpTwoWaySSL.java
@@ -17,6 +17,8 @@
 
 package org.apache.nifi.processors.standard;
 
+import org.apache.commons.lang3.SystemUtils;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 
 /**
@@ -29,6 +31,7 @@ public class TestInvokeHttpTwoWaySSL extends TestInvokeHttpSSL {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         // useful for verbose logging output
         // don't commit this with this property enabled, or any 'mvn test' will be really verbose
         // System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard", "debug");
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java
index b9276be..7f5e5d8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.processors.standard;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.flowfile.FlowFile;
@@ -28,7 +29,9 @@ import org.apache.nifi.processors.standard.util.FileInfo;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
@@ -104,6 +107,11 @@ public class TestListFile {
         }
     };
 
+    @BeforeClass
+    public static void setupClass() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setUp() throws Exception {
         processor = new ListFile();
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenHTTP.java
index 0dee76c..9cc828d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenHTTP.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.standard;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.remote.io.socket.NetworkUtils;
@@ -29,7 +30,9 @@ import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
@@ -79,6 +82,11 @@ public class TestListenHTTP {
 
     private int availablePort;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws IOException {
         proc = new ListenHTTP();
@@ -92,7 +100,7 @@ public class TestListenHTTP {
     @After
     public void teardown() {
         proc.shutdownHttpServer();
-        new File("/Users/alopresto/Workspace/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/my-file-text.txt").delete();
+        new File("my-file-text.txt").delete();
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutFile.java
index 0c202c1..a6ea1ce 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutFile.java
@@ -16,12 +16,15 @@
  */
 package org.apache.nifi.processors.standard;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.File;
@@ -45,6 +48,11 @@ public class TestPutFile {
     public static final String TARGET_DIRECTORY = "target/put-file";
     private File targetDir;
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void prepDestDirectory() throws IOException {
         targetDir = new File(TARGET_DIRECTORY);
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java
index aede7ac..c519869 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestUpdateRecord.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.processors.standard;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.json.JsonRecordSetWriter;
 import org.apache.nifi.json.JsonTreeReader;
 import org.apache.nifi.reporting.InitializationException;
@@ -29,7 +30,9 @@ import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -46,6 +49,12 @@ public class TestUpdateRecord {
     private MockRecordParser readerService;
     private MockRecordWriter writerService;
 
+    //Apparently pretty printing is not portable as these tests fail on windows
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws InitializationException {
         readerService = new MockRecordParser();
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
index 39b96bf..7842a16 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
@@ -62,7 +62,6 @@ public abstract class TestInvokeHttpCommon {
 
     public TestRunner runner;
 
-
     public void addHandler(Handler handler) {
         server.addHandler(handler);
     }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestPutTCPCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestPutTCPCommon.java
index be52478..5cf0e4a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestPutTCPCommon.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestPutTCPCommon.java
@@ -18,12 +18,15 @@
 package org.apache.nifi.processors.standard.util;
 
 import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.processors.standard.PutTCP;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.After;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -71,6 +74,11 @@ public abstract class TestPutTCPCommon {
     private final static String[] EMPTY_FILE = { "" };
     private final static String[] VALID_FILES = { "abcdefghijklmnopqrstuvwxyz", "zyxwvutsrqponmlkjihgfedcba", "12345678", "343424222", "!@£$%^&*()_+:|{}[];\\" };
 
+    @BeforeClass
+    public static void setUpSuite() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() throws Exception {
         recvQueue = new ArrayBlockingQueue<List<Byte>>(BUFFER_SIZE);
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java
index dd10550..3f7c82c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPKeyBasedEncryptorTest.java
@@ -26,6 +26,7 @@ import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.security.Security;
 import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.security.util.EncryptionMethod;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
@@ -33,6 +34,7 @@ import org.bouncycastle.openpgp.PGPEncryptedData;
 import org.bouncycastle.openpgp.PGPUtil;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -54,6 +56,7 @@ public class OpenPGPKeyBasedEncryptorTest {
 
     @BeforeClass
     public static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         Security.addProvider(new BouncyCastleProvider());
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java
index 3ad926b..fdd330b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/security/util/crypto/OpenPGPPasswordBasedEncryptorTest.java
@@ -26,6 +26,7 @@ import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.security.Security;
 import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.security.util.EncryptionMethod;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
@@ -33,6 +34,7 @@ import org.bouncycastle.openpgp.PGPEncryptedData;
 import org.bouncycastle.openpgp.PGPUtil;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -50,6 +52,7 @@ public class OpenPGPPasswordBasedEncryptorTest {
 
     @BeforeClass
     public static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         Security.addProvider(new BouncyCastleProvider());
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
index 75447f2..b941c09 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.json;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.record.NullSuppression;
 import org.apache.nifi.schema.access.SchemaNameAsAttribute;
@@ -111,10 +112,10 @@ public class TestWriteJsonResult {
             writer.write(rs);
         }
 
-        final String output = baos.toString();
+        final String output = baos.toString("UTF-8");
 
         final String expected = new String(Files.readAllBytes(Paths.get("src/test/resources/json/output/dataTypes.json")));
-        assertEquals(expected, output);
+        assertEquals(StringUtils.deleteWhitespace(expected), StringUtils.deleteWhitespace(output));
     }
 
 
diff --git a/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/test/java/org/apache/nifi/processors/stateful/analysis/TestAttributeRollingWindow.java b/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/test/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindowIT.java
similarity index 99%
rename from nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/test/java/org/apache/nifi/processors/stateful/analysis/TestAttributeRollingWindow.java
rename to nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/test/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindowIT.java
index 6b82eb8..8a24956 100644
--- a/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/test/java/org/apache/nifi/processors/stateful/analysis/TestAttributeRollingWindow.java
+++ b/nifi-nar-bundles/nifi-stateful-analysis-bundle/nifi-stateful-analysis-processors/src/test/java/org/apache/nifi/processors/stateful/analysis/AttributeRollingWindowIT.java
@@ -35,7 +35,7 @@ import static org.apache.nifi.processors.stateful.analysis.AttributeRollingWindo
 import static org.apache.nifi.processors.stateful.analysis.AttributeRollingWindow.ROLLING_WINDOW_VALUE_KEY;
 import static org.junit.Assume.assumeFalse;
 
-public class TestAttributeRollingWindow {
+public class AttributeRollingWindowIT {
 
     @Test
     public void testFailureDueToBadAttribute() throws InterruptedException {
diff --git a/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/configmigrator/ConfigMigratorSpec.groovy b/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/configmigrator/ConfigMigratorSpec.groovy
index fb82a22..54b6199 100644
--- a/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/configmigrator/ConfigMigratorSpec.groovy
+++ b/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/configmigrator/ConfigMigratorSpec.groovy
@@ -17,9 +17,10 @@
 
 package org.apache.nifi.toolkit.admin.configmigrator
 
-import groovy.xml.XmlUtil
 import org.apache.commons.io.FileUtils
 import org.apache.commons.lang3.SystemUtils
+import org.junit.Assume
+import org.junit.BeforeClass
 import org.junit.Rule
 import org.junit.contrib.java.lang.system.SystemOutRule
 import spock.lang.Specification
@@ -36,6 +37,10 @@ class ConfigMigratorSpec extends Specification{
     @Rule
     public final SystemOutRule systemOutRule = new SystemOutRule().enableLog()
 
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
+    }
 
     def "get rules directory name"(){
 
diff --git a/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/filemanager/FileManagerToolSpec.groovy b/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/filemanager/FileManagerToolSpec.groovy
index 1e19bad..959fa56 100644
--- a/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/filemanager/FileManagerToolSpec.groovy
+++ b/nifi-toolkit/nifi-toolkit-admin/src/test/groovy/org/apache/nifi/toolkit/admin/filemanager/FileManagerToolSpec.groovy
@@ -19,6 +19,8 @@ package org.apache.nifi.toolkit.admin.filemanager
 import org.apache.commons.cli.ParseException
 import org.apache.commons.io.FileUtils
 import org.apache.commons.lang3.SystemUtils
+import org.junit.Assume
+import org.junit.BeforeClass
 import org.junit.Rule
 import org.junit.contrib.java.lang.system.ExpectedSystemExit
 import org.junit.contrib.java.lang.system.SystemOutRule
@@ -34,6 +36,10 @@ class FileManagerToolSpec extends Specification{
     @Rule
     public final SystemOutRule systemOutRule = new SystemOutRule().enableLog()
 
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
+    }
 
     def "print help and usage info"() {
 
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/TestCLICompleter.java b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/TestCLICompleter.java
index 3e2e2a3..35e1ee0 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/TestCLICompleter.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/TestCLICompleter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.toolkit.cli;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.registry.client.NiFiRegistryClient;
 import org.apache.nifi.toolkit.cli.api.ClientFactory;
 import org.apache.nifi.toolkit.cli.api.Command;
@@ -33,6 +34,7 @@ import org.apache.nifi.toolkit.cli.impl.session.SessionVariable;
 import org.jline.reader.Candidate;
 import org.jline.reader.LineReader;
 import org.jline.reader.impl.DefaultParser;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -53,6 +55,7 @@ public class TestCLICompleter {
 
     @BeforeClass
     public static void setupCompleter() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
         final Session session = new InMemorySession();
         final ClientFactory<NiFiClient> niFiClientFactory = new NiFiClientFactory();
         final ClientFactory<NiFiRegistryClient> nifiRegClientFactory = new NiFiRegistryClientFactory();
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestBucketsResult.java b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestBucketsResult.java
index f535f64..2933153 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestBucketsResult.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestBucketsResult.java
@@ -16,11 +16,14 @@
  */
 package org.apache.nifi.toolkit.cli.impl.result;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.registry.bucket.Bucket;
 import org.apache.nifi.toolkit.cli.api.ResultType;
 import org.apache.nifi.toolkit.cli.impl.result.registry.BucketsResult;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
@@ -36,6 +39,11 @@ public class TestBucketsResult {
     private ByteArrayOutputStream outputStream;
     private PrintStream printStream;
 
+    @BeforeClass
+    public static void setupCompleter() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         this.outputStream = new ByteArrayOutputStream();
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java
index 5421ea4..ed2f1f4 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java
@@ -16,13 +16,16 @@
  */
 package org.apache.nifi.toolkit.cli.impl.result;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.toolkit.cli.api.ResultType;
 import org.apache.nifi.toolkit.cli.impl.result.nifi.RegistryClientsResult;
 import org.apache.nifi.web.api.dto.RegistryDTO;
 import org.apache.nifi.web.api.entity.RegistryClientEntity;
 import org.apache.nifi.web.api.entity.RegistryClientsEntity;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
@@ -38,6 +41,11 @@ public class TestRegistryClientResult {
     private ByteArrayOutputStream outputStream;
     private PrintStream printStream;
 
+    @BeforeClass
+    public static void setupCompleter() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         this.outputStream = new ByteArrayOutputStream();
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowSnapshotMetadataResult.java b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowSnapshotMetadataResult.java
index 3fb98d2..b3e3a1d 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowSnapshotMetadataResult.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowSnapshotMetadataResult.java
@@ -16,11 +16,14 @@
  */
 package org.apache.nifi.toolkit.cli.impl.result;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.toolkit.cli.api.ResultType;
 import org.apache.nifi.toolkit.cli.impl.result.registry.VersionedFlowSnapshotMetadataResult;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
@@ -37,6 +40,11 @@ public class TestVersionedFlowSnapshotMetadataResult {
     private ByteArrayOutputStream outputStream;
     private PrintStream printStream;
 
+    @BeforeClass
+    public static void setupCompleter() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         this.outputStream = new ByteArrayOutputStream();
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowsResult.java b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowsResult.java
index f6951b7..d61fa82 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowsResult.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestVersionedFlowsResult.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.toolkit.cli.impl.result;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.toolkit.cli.api.Context;
 import org.apache.nifi.toolkit.cli.api.ReferenceResolver;
@@ -23,7 +24,9 @@ import org.apache.nifi.toolkit.cli.api.ResultType;
 import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
 import org.apache.nifi.toolkit.cli.impl.result.registry.VersionedFlowsResult;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -41,6 +44,11 @@ public class TestVersionedFlowsResult {
     private PrintStream printStream;
     private List<VersionedFlow> flows;
 
+    @BeforeClass
+    public static void setupCompleter() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         this.outputStream = new ByteArrayOutputStream();
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/writer/TestDynamicTableWriter.java b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/writer/TestDynamicTableWriter.java
index 49e9a64..39e3ca6 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/writer/TestDynamicTableWriter.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/writer/TestDynamicTableWriter.java
@@ -16,8 +16,11 @@
  */
 package org.apache.nifi.toolkit.cli.impl.result.writer;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
@@ -32,6 +35,11 @@ public class TestDynamicTableWriter {
     private ByteArrayOutputStream outputStream;
     private PrintStream printStream;
 
+    @BeforeClass
+    public static void setupCompleter() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     public void setup() {
         this.table = new Table.Builder()
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
index 317c779..edc6aa7 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
@@ -27,6 +27,7 @@ import org.apache.nifi.toolkit.tls.commandLine.CommandLineParseException
 import org.apache.nifi.util.NiFiProperties
 import org.apache.nifi.util.console.TextDevice
 import org.apache.nifi.util.console.TextDevices
+import org.apache.nifi.util.file.FileUtils
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After
 import org.junit.AfterClass
@@ -113,6 +114,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
 
     @BeforeClass
     static void setUpOnce() throws Exception {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
         Security.addProvider(new BouncyCastleProvider())
 
         logger.metaClass.methodMissing = { String name, args ->
@@ -811,7 +813,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS)
 
         File inputPropertiesFile = new File("src/test/resources/nifi_with_sensitive_properties_unprotected.properties")
-        File workingFile = new File("tmp_nifi.properties")
+        File workingFile = new File("target/tmp_nifi.properties")
         workingFile.delete()
 
         Files.copy(inputPropertiesFile.toPath(), workingFile.toPath())
@@ -843,7 +845,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         Assume.assumeTrue("Test only runs on Windows", SystemUtils.IS_OS_WINDOWS)
 
         File inputPropertiesFile = new File("src/test/resources/nifi_with_sensitive_properties_unprotected.properties")
-        File workingFile = new File("tmp_nifi.properties")
+        File workingFile = new File("target/tmp_nifi.properties")
         workingFile.delete()
 
         Files.copy(inputPropertiesFile.toPath(), workingFile.toPath())
@@ -993,7 +995,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
     void testShouldWriteKeyToBootstrapConf() {
         // Arrange
         File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
-        File workingFile = new File("tmp_bootstrap.conf")
+        File workingFile = new File("target/tmp_bootstrap.conf")
         workingFile.delete()
 
         Files.copy(emptyKeyFile.toPath(), workingFile.toPath())
@@ -1022,11 +1024,12 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         workingFile.deleteOnExit()
     }
 
+    @Ignore("this test needs to be updated to ensure any created files are done under target")
     @Test
     void testWriteKeyToBootstrapConfShouldHandleReadFailure() {
         // Arrange
         File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
-        File workingFile = new File("tmp_bootstrap.conf")
+        File workingFile = new File("target/tmp_bootstrap.conf")
         workingFile.delete()
 
         Files.copy(emptyKeyFile.toPath(), workingFile.toPath())
@@ -1051,11 +1054,12 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         workingFile.deleteOnExit()
     }
 
+    @Ignore("this test needs to be updated to ensure any created files are done under target")
     @Test
     void testWriteKeyToBootstrapConfShouldHandleWriteFailure() {
         // Arrange
         File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
-        File workingFile = new File("tmp_bootstrap.conf")
+        File workingFile = new File("target/tmp_bootstrap.conf")
         workingFile.delete()
 
         Files.copy(emptyKeyFile.toPath(), workingFile.toPath())
@@ -1315,7 +1319,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
     void testShouldWriteNiFiProperties() {
         // Arrange
         File inputPropertiesFile = new File("src/test/resources/nifi_with_sensitive_properties_unprotected.properties")
-        File workingFile = new File("tmp_nifi.properties")
+        File workingFile = new File("target/tmp_nifi.properties")
         workingFile.delete()
 
         final List<String> originalLines = inputPropertiesFile.readLines()
@@ -1356,7 +1360,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
     void testShouldWriteNiFiPropertiesInSameLocation() {
         // Arrange
         File inputPropertiesFile = new File("src/test/resources/nifi_with_sensitive_properties_unprotected.properties")
-        File workingFile = new File("tmp_nifi.properties")
+        File workingFile = new File("target/tmp_nifi.properties")
         workingFile.delete()
         Files.copy(inputPropertiesFile.toPath(), workingFile.toPath())
 
@@ -1402,7 +1406,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
     void testWriteNiFiPropertiesShouldHandleWriteFailureWhenFileExists() {
         // Arrange
         File inputPropertiesFile = new File("src/test/resources/nifi_with_sensitive_properties_unprotected.properties")
-        File workingFile = new File("tmp_nifi.properties")
+        File workingFile = new File("target/tmp_nifi.properties")
         workingFile.delete()
 
         Files.copy(inputPropertiesFile.toPath(), workingFile.toPath())
diff --git a/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml b/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml
index 802a831..2bb3128 100644
--- a/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml
+++ b/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml
@@ -23,5 +23,11 @@
             <artifactId>nifi-utils</artifactId>
             <version>1.11.4-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.9</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java b/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java
index 7f8a7a1..d01e3b6 100644
--- a/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java
+++ b/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java
@@ -17,8 +17,10 @@
 
 package org.apache.nifi.toolkit.repos.flowfile;
 
+import org.apache.commons.lang3.SystemUtils;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 
 import static org.junit.Assert.assertEquals;
@@ -28,11 +30,17 @@ import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestRepairCorruptedFileEndings {
     private final File targetFile = new File("target/1.bin");
 
+    @BeforeClass
+    public static void setupInstance() {
+        Assume.assumeTrue("Test only runs on *nix", !SystemUtils.IS_OS_WINDOWS);
+    }
+
     @Before
     @After
     public void cleanup() {


[nifi] 10/47: NIFI-7121 Updated comment to state a 'static' salt is used in the constructor. (#4098)

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 3106604ae95dcb0e4bb08857afc13a5d4efa377a
Author: M Tien <56...@users.noreply.github.com>
AuthorDate: Tue Mar 3 15:50:49 2020 -0800

    NIFI-7121 Updated comment to state a 'static' salt is used in the constructor. (#4098)
    
    Signed-off-by: Andy LoPresto <al...@apache.org>
---
 .../java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
index 0697f3d..22e618c 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
@@ -70,7 +70,7 @@ public class Argon2SecureHasher implements SecureHasher {
     }
 
     /**
-     * Instantiates an Argon2 secure hasher using the provided cost parameters. A unique
+     * Instantiates an Argon2 secure hasher using the provided cost parameters. A static
      * {@link #DEFAULT_SALT_LENGTH} byte salt will be generated on every hash request.
      *
      * @param hashLength  the output length in bytes ({@code 4 to 2^32 - 1})


[nifi] 25/47: NIFI-7200: Revert "NIFI-6530 - HTTP SiteToSite server returns 201 in case no data is available"

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6f5bd1778ee31af2fb3190ccdb2908f4495ac3ad
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Tue Mar 10 11:56:49 2020 -0400

    NIFI-7200: Revert "NIFI-6530 - HTTP SiteToSite server returns 201 in case no data is available"
    
    This reverts commit f01668e66ad2e45197915769e966a4be27e1592e.
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../apache/nifi/remote/client/PeerSelector.java    | 22 +++++---
 .../apache/nifi/remote/client/http/HttpClient.java | 22 ++------
 .../http/TransportProtocolVersionNegotiator.java   |  1 -
 .../client/socket/EndpointConnectionPool.java      | 19 +++----
 .../nifi/remote/client/socket/SocketClient.java    |  6 +--
 .../nifi/remote/exception/NoContentException.java  | 39 --------------
 .../remote/exception/NoValidPeerException.java     | 40 --------------
 .../protocol/socket/SocketClientTransaction.java   |  4 --
 .../nifi/remote/util/SiteToSiteRestApiClient.java  |  6 +--
 .../nifi/remote/client/TestPeerSelector.java       | 31 +++++------
 .../nifi/remote/client/http/TestHttpClient.java    | 63 +++-------------------
 .../socket/TestSocketClientTransaction.java        | 17 +++---
 .../java/org/apache/nifi/spark/NiFiReceiver.java   |  7 ---
 .../nifi/remote/StandardRemoteGroupPort.java       | 13 ++---
 .../stateless/core/StatelessRemoteOutputPort.java  |  8 +--
 .../apache/nifi/web/api/DataTransferResource.java  | 19 +------
 .../apache/nifi/toolkit/s2s/SiteToSiteCliMain.java |  3 --
 .../nifi/toolkit/s2s/SiteToSiteReceiver.java       |  4 --
 18 files changed, 72 insertions(+), 252 deletions(-)

diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/PeerSelector.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/PeerSelector.java
index 8235a38..0a61077 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/PeerSelector.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/PeerSelector.java
@@ -224,7 +224,7 @@ public class PeerSelector {
      *                  for RECEIVE, a peer with more flow files is preferred
      * @return a selected peer, if there is no available peer or all peers are penalized, then return null
      */
-    public ArrayList<PeerStatus> getPeerStatuses(final TransferDirection direction) {
+    public PeerStatus getNextPeerStatus(final TransferDirection direction) {
         List<PeerStatus> peerList = peerStatuses;
         if (isPeerRefreshNeeded(peerList)) {
             peerRefreshLock.lock();
@@ -251,15 +251,25 @@ public class PeerSelector {
             }
         }
 
-
         if (peerList == null || peerList.isEmpty()) {
-            return new ArrayList<PeerStatus>();
+            return null;
         }
 
-        ArrayList<PeerStatus> retVal = new ArrayList<>(peerList);
-        retVal.removeIf(p -> isPenalized(p));
+        PeerStatus peerStatus;
+        for (int i = 0; i < peerList.size(); i++) {
+            final long idx = peerIndex.getAndIncrement();
+            final int listIndex = (int) (idx % peerList.size());
+            peerStatus = peerList.get(listIndex);
+
+            if (isPenalized(peerStatus)) {
+                logger.debug("{} {} is penalized; will not communicate with this peer", this, peerStatus);
+            } else {
+                return peerStatus;
+            }
+        }
 
-        return retVal;
+        logger.debug("{} All peers appear to be penalized; returning null", this);
+        return null;
     }
 
     private List<PeerStatus> createPeerStatusList(final TransferDirection direction) throws IOException {
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java
index 690cdfd..660f5ea 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java
@@ -27,8 +27,6 @@ import org.apache.nifi.remote.client.PeerSelector;
 import org.apache.nifi.remote.client.PeerStatusProvider;
 import org.apache.nifi.remote.client.SiteToSiteClientConfig;
 import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.NoContentException;
-import org.apache.nifi.remote.exception.NoValidPeerException;
 import org.apache.nifi.remote.exception.PortNotRunningException;
 import org.apache.nifi.remote.exception.ProtocolException;
 import org.apache.nifi.remote.exception.UnknownPortException;
@@ -43,7 +41,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -128,11 +125,9 @@ public class HttpClient extends AbstractSiteToSiteClient implements PeerStatusPr
     @Override
     public Transaction createTransaction(final TransferDirection direction) throws HandshakeException, PortNotRunningException, ProtocolException, UnknownPortException, IOException {
         final int timeoutMillis = (int) config.getTimeout(TimeUnit.MILLISECONDS);
-        Integer peersWithNoContent = 0;
 
-        ArrayList<PeerStatus> peers = peerSelector.getPeerStatuses(direction);
-
-        for  (PeerStatus peerStatus : peers) {
+        PeerStatus peerStatus;
+        while ((peerStatus = peerSelector.getNextPeerStatus(direction)) != null) {
             logger.debug("peerStatus={}", peerStatus);
 
             final CommunicationsSession commSession = new HttpCommunicationsSession();
@@ -174,11 +169,6 @@ public class HttpClient extends AbstractSiteToSiteClient implements PeerStatusPr
             try {
                 transactionUrl = apiClient.initiateTransaction(direction, portId);
                 commSession.setUserDn(apiClient.getTrustedPeerDn());
-            } catch (final NoContentException e) {
-                apiClient.close();
-                peersWithNoContent++;
-                logger.debug("Peer {} has no flowfiles to provide", peer);
-                continue;
             } catch (final Exception e) {
                 apiClient.close();
                 logger.warn("Penalizing a peer {} due to {}", peer, e.toString());
@@ -221,12 +211,8 @@ public class HttpClient extends AbstractSiteToSiteClient implements PeerStatusPr
             return transaction;
         }
 
-        if(peersWithNoContent > 0) {
-            return null;
-        }
-        String error = new String("Couldn't find a valid peer to communicate with.");
-        logger.info(error);
-        throw new NoValidPeerException(error);
+        logger.info("Couldn't find a valid peer to communicate with.");
+        return null;
     }
 
     private String resolveNodeApiUrl(final PeerDescription description) {
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/TransportProtocolVersionNegotiator.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/TransportProtocolVersionNegotiator.java
index 844a92e..d0a6368 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/TransportProtocolVersionNegotiator.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/TransportProtocolVersionNegotiator.java
@@ -32,7 +32,6 @@ public class TransportProtocolVersionNegotiator extends StandardVersionNegotiato
     public int getTransactionProtocolVersion() {
         switch (getVersion()) {
             case 1:
-            case 2:
                 return 5;
             default:
                 throw new RuntimeException("Transport protocol version " + getVersion()
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index 53bd963..0cf1b53 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -30,7 +30,6 @@ import org.apache.nifi.remote.client.SiteInfoProvider;
 import org.apache.nifi.remote.client.SiteToSiteClientConfig;
 import org.apache.nifi.remote.codec.FlowFileCodec;
 import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.NoValidPeerException;
 import org.apache.nifi.remote.exception.PortNotRunningException;
 import org.apache.nifi.remote.exception.TransmissionDisabledException;
 import org.apache.nifi.remote.exception.UnknownPortException;
@@ -166,9 +165,14 @@ public class EndpointConnectionPool implements PeerStatusProvider {
             throw new UnreachableClusterException("Unable to refresh details from any of the configured remote instances.", ioe);
         }
 
-        for  (PeerStatus peerStatus : peerSelector.getPeerStatuses(direction)) {
+        do {
             final List<EndpointConnection> addBack = new ArrayList<>();
+            logger.debug("{} getting next peer status", this);
+            final PeerStatus peerStatus = peerSelector.getNextPeerStatus(direction);
             logger.debug("{} next peer status = {}", this, peerStatus);
+            if (peerStatus == null) {
+                return null;
+            }
 
             final PeerDescription peerDescription = peerStatus.getPeerDescription();
             BlockingQueue<EndpointConnection> connectionQueue = connectionQueueMap.get(peerDescription);
@@ -188,7 +192,7 @@ public class EndpointConnectionPool implements PeerStatusProvider {
                 if (connection == null && !addBack.isEmpty()) {
                     // all available connections have been penalized.
                     logger.debug("{} all Connections for {} are penalized; returning no Connection", this, portId);
-                    throw new NoValidPeerException("All peers are penalized");
+                    return null;
                 }
 
                 if (connection != null && connection.getPeer().isPenalized(portId)) {
@@ -314,13 +318,10 @@ public class EndpointConnectionPool implements PeerStatusProvider {
                 }
             }
 
-            if( connection != null && codec != null && commsSession != null && protocol != null) {
-                activeConnections.add(connection);
-                return connection;
-            }
-        }
-        throw new NoValidPeerException("Didn't find any valid peer to connect to");
+        } while (connection == null || codec == null || commsSession == null || protocol == null);
 
+        activeConnections.add(connection);
+        return connection;
     }
 
     public boolean offer(final EndpointConnection endpointConnection) {
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
index ff8e0d6..64a174a 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -23,7 +23,6 @@ import org.apache.nifi.remote.TransactionCompletion;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.AbstractSiteToSiteClient;
 import org.apache.nifi.remote.client.SiteToSiteClientConfig;
-import org.apache.nifi.remote.exception.NoContentException;
 import org.apache.nifi.remote.protocol.DataPacket;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -126,13 +125,14 @@ public class SocketClient extends AbstractSiteToSiteClient {
         }
 
         final EndpointConnection connectionState = pool.getEndpointConnection(direction, getConfig());
+        if (connectionState == null) {
+            return null;
+        }
 
         final Transaction transaction;
         try {
             transaction = connectionState.getSocketClientProtocol().startTransaction(
                     connectionState.getPeer(), connectionState.getCodec(), direction);
-        } catch (final NoContentException e) {
-            return null;
         } catch (final Throwable t) {
             pool.terminate(connectionState);
             throw new IOException("Unable to create Transaction to communicate with " + connectionState.getPeer(), t);
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoContentException.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoContentException.java
deleted file mode 100644
index a0dd23d..0000000
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoContentException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-import java.io.IOException;
-
-/**
- * A NoContentException occurs when the remote peer has no flowfiles to provide
- */
-public class NoContentException extends IOException {
-
-    private static final long serialVersionUID = -689032011082690815L;
-
-    public NoContentException(final String message, final Throwable cause) {
-        super(message, cause);
-    }
-
-    public NoContentException(final String message) {
-        super(message);
-    }
-
-    public NoContentException(final Throwable cause) {
-        super(cause);
-    }
-}
\ No newline at end of file
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoValidPeerException.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoValidPeerException.java
deleted file mode 100644
index 30a51a0..0000000
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NoValidPeerException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-import java.io.IOException;
-
-
-/**
- * A NoValidPeerException occurs when all the remote peers are penalized or none exists
- */
-public class NoValidPeerException extends IOException {
-
-    private static final long serialVersionUID = 8421102798129193880L;
-
-    public NoValidPeerException(final String message, final Throwable cause) {
-        super(message, cause);
-    }
-
-    public NoValidPeerException(final String message) {
-        super(message);
-    }
-
-    public NoValidPeerException(final Throwable cause) {
-        super(cause);
-    }
-}
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
index 85d6c1a..8b68c9e 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
@@ -21,7 +21,6 @@ import org.apache.nifi.remote.AbstractTransaction;
 import org.apache.nifi.remote.Peer;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.codec.FlowFileCodec;
-import org.apache.nifi.remote.exception.NoContentException;
 import org.apache.nifi.remote.exception.ProtocolException;
 import org.apache.nifi.remote.protocol.RequestType;
 import org.apache.nifi.remote.protocol.Response;
@@ -44,9 +43,6 @@ public class SocketClientTransaction extends AbstractTransaction {
         this.dos = new DataOutputStream(peer.getCommunicationsSession().getOutput().getOutputStream());
 
         initialize();
-        if (direction == TransferDirection.RECEIVE && !this.dataAvailable){
-            throw new NoContentException("Remote side has no flowfiles to provide");
-        }
     }
 
     private void initialize() throws IOException {
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java
index 3270708..249325d 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java
@@ -64,7 +64,6 @@ import org.apache.nifi.remote.Peer;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.http.TransportProtocolVersionNegotiator;
 import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.NoContentException;
 import org.apache.nifi.remote.exception.PortNotRunningException;
 import org.apache.nifi.remote.exception.ProtocolException;
 import org.apache.nifi.remote.exception.UnknownPortException;
@@ -148,7 +147,6 @@ public class SiteToSiteRestApiClient implements Closeable {
     private static final int RESPONSE_CODE_OK = 200;
     private static final int RESPONSE_CODE_CREATED = 201;
     private static final int RESPONSE_CODE_ACCEPTED = 202;
-    private static final int RESPONSE_CODE_NO_CONTENT = 204;
     private static final int RESPONSE_CODE_BAD_REQUEST = 400;
     private static final int RESPONSE_CODE_FORBIDDEN = 403;
     private static final int RESPONSE_CODE_NOT_FOUND = 404;
@@ -173,7 +171,7 @@ public class SiteToSiteRestApiClient implements Closeable {
     private int batchCount = 0;
     private long batchSize = 0;
     private long batchDurationMillis = 0;
-    private TransportProtocolVersionNegotiator transportProtocolVersionNegotiator = new TransportProtocolVersionNegotiator(2,1);
+    private TransportProtocolVersionNegotiator transportProtocolVersionNegotiator = new TransportProtocolVersionNegotiator(1);
 
     private String trustedPeerDn;
     private final ScheduledExecutorService ttlExtendTaskExecutor;
@@ -500,8 +498,6 @@ public class SiteToSiteRestApiClient implements Closeable {
                 }
                 serverTransactionTtl = Integer.parseInt(serverTransactionTtlHeader.getValue());
                 break;
-            case RESPONSE_CODE_NO_CONTENT:
-                throw new NoContentException("Server has no flowfiles to provide");
 
             default:
                 try (InputStream content = response.getEntity().getContent()) {
diff --git a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/TestPeerSelector.java b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/TestPeerSelector.java
index 72dd9a6..d98774e 100644
--- a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/TestPeerSelector.java
+++ b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/TestPeerSelector.java
@@ -35,7 +35,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -50,6 +49,8 @@ import static java.util.stream.Collectors.groupingBy;
 import static java.util.stream.Collectors.reducing;
 import static java.util.stream.Collectors.toMap;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.any;
@@ -227,13 +228,10 @@ public class TestPeerSelector {
             throw new IOException("Connection refused. " + peerFetchStatusesFrom + " is not running.");
         }).when(peerStatusProvider).fetchRemotePeerStatuses(any(PeerDescription.class));
 
-
-        ArrayList<PeerStatus> peers;
-
         // 1st attempt. It uses the bootstrap node.
         peerSelector.refreshPeers();
-        peers = peerSelector.getPeerStatuses(TransferDirection.RECEIVE);
-        assert(!peers.isEmpty());
+        PeerStatus peerStatus = peerSelector.getNextPeerStatus(TransferDirection.RECEIVE);
+        assertNotNull(peerStatus);
 
         // Proceed time so that peer selector refresh statuses.
         peerStatuses.remove(bootstrapNodeStatus);
@@ -241,35 +239,34 @@ public class TestPeerSelector {
 
         // 2nd attempt.
         peerSelector.refreshPeers();
-        peers = peerSelector.getPeerStatuses(TransferDirection.RECEIVE);
-        assert(!peers.isEmpty());
-        assertEquals("Node2 should be returned since node 2 is the only available node.", node2, peers.get(0).getPeerDescription());
+        peerStatus = peerSelector.getNextPeerStatus(TransferDirection.RECEIVE);
+        assertNotNull(peerStatus);
+        assertEquals("Node2 should be returned since node 2 is the only available node.", node2, peerStatus.getPeerDescription());
 
         // Proceed time so that peer selector refresh statuses.
         systemTime.offset += TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES) + 1;
 
         // 3rd attempt.
         peerSelector.refreshPeers();
-        peers = peerSelector.getPeerStatuses(TransferDirection.RECEIVE);
-        assert(!peers.isEmpty());
-        assertEquals("Node2 should be returned since node 2 is the only available node.", node2, peers.get(0).getPeerDescription());
+        peerStatus = peerSelector.getNextPeerStatus(TransferDirection.RECEIVE);
+        assertNotNull(peerStatus);
+        assertEquals("Node2 should be returned since node 2 is the only available node.", node2, peerStatus.getPeerDescription());
 
         // Remove node2 to simulate that it goes down. There's no available node at this point.
         peerStatuses.remove(node2Status);
         systemTime.offset += TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES) + 1;
 
         peerSelector.refreshPeers();
-        peers = peerSelector.getPeerStatuses(TransferDirection.RECEIVE);
-        assertTrue("PeerSelector should return an empty list as next peer statuses, since there's no available peer", peers.isEmpty());
+        peerStatus = peerSelector.getNextPeerStatus(TransferDirection.RECEIVE);
+        assertNull("PeerSelector should return null as next peer status, since there's no available peer", peerStatus);
 
         // Add node1 back. PeerSelector should be able to fetch peer statuses because it always tries to fetch at least from the bootstrap node.
         peerStatuses.add(bootstrapNodeStatus);
         systemTime.offset += TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES) + 1;
 
         peerSelector.refreshPeers();
-        peers = peerSelector.getPeerStatuses(TransferDirection.RECEIVE);
-        assert(!peers.isEmpty());
-        assertEquals("Node1 should be returned since node 1 is the only available node.", bootstrapNode, peers.get(0).getPeerDescription());
+        peerStatus = peerSelector.getNextPeerStatus(TransferDirection.RECEIVE);
+        assertEquals("Node1 should be returned since node 1 is the only available node.", bootstrapNode, peerStatus.getPeerDescription());
     }
 
     @Test
diff --git a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
index 706b4ca..ded1db1 100644
--- a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
@@ -57,7 +57,6 @@ import org.apache.nifi.remote.client.KeystoreType;
 import org.apache.nifi.remote.client.SiteToSiteClient;
 import org.apache.nifi.remote.codec.StandardFlowFileCodec;
 import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.NoValidPeerException;
 import org.apache.nifi.remote.io.CompressionInputStream;
 import org.apache.nifi.remote.io.CompressionOutputStream;
 import org.apache.nifi.remote.protocol.DataPacket;
@@ -198,21 +197,6 @@ public class TestHttpClient {
 
     }
 
-    public static class EmptyPortTransactionsServlet extends PortTransactionsServlet {
-
-        @Override
-        protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
-
-            final int reqProtocolVersion = getReqProtocolVersion(req);
-            if (reqProtocolVersion == 1) {
-                super.doPost(req, resp);
-            } else {
-                respondWithText(resp, "No flowfiles available", 204);
-            }
-        }
-
-    }
-
     public static class PortTransactionsAccessDeniedServlet extends HttpServlet {
 
         @Override
@@ -513,8 +497,6 @@ public class TestHttpClient {
         servletHandler.addServletWithMapping(OutputPortTransactionServlet.class, "/data-transfer/output-ports/output-timeout-data-ex-id/transactions/transaction-id");
         servletHandler.addServletWithMapping(FlowFilesTimeoutAfterDataExchangeServlet.class, "/data-transfer/output-ports/output-timeout-data-ex-id/transactions/transaction-id/flow-files");
 
-        servletHandler.addServletWithMapping(EmptyPortTransactionsServlet.class,"/data-transfer/output-ports/empty-output-running-id/transactions");
-
         server.start();
 
         logger.info("Starting server on port {} for HTTP, and {} for HTTPS", httpConnector.getLocalPort(), sslConnector.getLocalPort());
@@ -674,13 +656,6 @@ public class TestHttpClient {
         runningOutputPort.setState(ScheduledState.RUNNING.name());
         outputPorts.add(runningOutputPort);
 
-        final PortDTO emptyRunningOutputPort = new PortDTO();
-        emptyRunningOutputPort.setName("empty-output-running");
-        emptyRunningOutputPort.setId("empty-output-running-id");
-        emptyRunningOutputPort.setType("OUTPUT_PORT");
-        emptyRunningOutputPort.setState(ScheduledState.RUNNING.name());
-        outputPorts.add(emptyRunningOutputPort);
-
         final PortDTO timeoutOutputPort = new PortDTO();
         timeoutOutputPort.setName("output-timeout");
         timeoutOutputPort.setId("output-timeout-id");
@@ -743,10 +718,9 @@ public class TestHttpClient {
                 .build()
         ) {
             final Transaction transaction = client.createTransaction(TransferDirection.SEND);
-            fail();
 
-        } catch (final NoValidPeerException e) {
-            assertNotNull(e.getMessage());
+            assertNull(transaction);
+
         }
 
     }
@@ -763,10 +737,9 @@ public class TestHttpClient {
                 .build()
         ) {
             final Transaction transaction = client.createTransaction(TransferDirection.SEND);
-            fail();
 
-        } catch (final NoValidPeerException e) {
-            assertNotNull(e.getMessage());
+            assertNull(transaction);
+
         }
 
     }
@@ -782,11 +755,11 @@ public class TestHttpClient {
                 .build()
         ) {
             final Transaction transaction = client.createTransaction(TransferDirection.SEND);
-            fail();
 
-        } catch (final NoValidPeerException e) {
-            assertNotNull(e.getMessage());
+            assertNull(transaction);
+
         }
+
     }
 
     @Test
@@ -881,10 +854,7 @@ public class TestHttpClient {
                         .build()
         ) {
             final Transaction transaction = client.createTransaction(TransferDirection.SEND);
-            fail();
-
-        } catch (final NoValidPeerException e) {
-            assertNotNull("createTransaction should fail at peer selection and return null.", e.getMessage());
+            assertNull("createTransaction should fail at peer selection and return null.", transaction);
         }
 
     }
@@ -1254,23 +1224,6 @@ public class TestHttpClient {
         }
     }
 
-    @Test
-    public void testReceiveEmptyPort() throws Exception {
-
-        try (
-                SiteToSiteClient client = getDefaultBuilder()
-                        .portName("empty-output-running")
-                        .build()
-        ) {
-            try {
-                final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
-                assertNull(transaction);
-            } catch (IOException e) {
-                fail();
-            }
-        }
-    }
-
     private void testReceive(SiteToSiteClient client) throws IOException {
         final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
 
diff --git a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/protocol/socket/TestSocketClientTransaction.java b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/protocol/socket/TestSocketClientTransaction.java
index 048d612..edae052 100644
--- a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/protocol/socket/TestSocketClientTransaction.java
+++ b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/protocol/socket/TestSocketClientTransaction.java
@@ -23,7 +23,6 @@ import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.codec.FlowFileCodec;
 import org.apache.nifi.remote.codec.StandardFlowFileCodec;
-import org.apache.nifi.remote.exception.NoContentException;
 import org.apache.nifi.remote.io.socket.SocketCommunicationsSession;
 import org.apache.nifi.remote.io.socket.SocketInput;
 import org.apache.nifi.remote.io.socket.SocketOutput;
@@ -45,6 +44,7 @@ import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.createDataPack
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveOneFlowFile;
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveTwoFlowFiles;
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveWithInvalidChecksum;
+import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveZeroFlowFile;
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendButDestinationFull;
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendOneFlowFile;
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendTwoFlowFiles;
@@ -52,7 +52,6 @@ import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendWithIn
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendZeroFlowFile;
 import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.readContents;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -93,12 +92,14 @@ public class TestSocketClientTransaction {
         ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
 
-        try {
-            SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.RECEIVE);
-            fail();
-        } catch (final NoContentException e) {
-            assertEquals("Remote side has no flowfiles to provide", e.getMessage());
-        }
+        SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.RECEIVE);
+
+        execReceiveZeroFlowFile(transaction);
+
+        // Verify what client has sent.
+        DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
+        assertEquals(RequestType.RECEIVE_FLOWFILES, RequestType.readRequestType(sentByClient));
+        assertEquals(-1, sentByClient.read());
     }
 
     @Test
diff --git a/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java b/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java
index 278e6b6..83a7e42 100644
--- a/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java
+++ b/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java
@@ -147,13 +147,6 @@ public class NiFiReceiver extends Receiver<NiFiDataPacket> {
                 try {
                     while (!isStopped()) {
                         final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
-                        if (transaction == null) {
-                            try {
-                                Thread.sleep(1000L);
-                            } catch (InterruptedException e) {
-                            }
-                            continue;
-                        }
                         DataPacket dataPacket = transaction.receive();
                         if (dataPacket == null) {
                             transaction.confirm();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index 95f5c2e..da050e2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -36,7 +36,6 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.remote.client.SiteToSiteClient;
 import org.apache.nifi.remote.client.SiteToSiteClientConfig;
-import org.apache.nifi.remote.exception.NoValidPeerException;
 import org.apache.nifi.remote.exception.PortNotRunningException;
 import org.apache.nifi.remote.exception.ProtocolException;
 import org.apache.nifi.remote.exception.UnknownPortException;
@@ -223,13 +222,6 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         final Transaction transaction;
         try {
             transaction = client.createTransaction(transferDirection);
-        } catch (final NoValidPeerException e) {
-            final String message = String.format("%s Unable to create transaction to communicate with; all peers must be penalized, so yielding context", this);
-            logger.debug(message);
-            session.rollback();
-            context.yield();
-            remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
-            return;
         } catch (final PortNotRunningException e) {
             context.yield();
             this.targetRunning.set(false);
@@ -265,10 +257,11 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
             return;
         }
+
         if (transaction == null) {
+            logger.debug("{} Unable to create transaction to communicate with; all peers must be penalized, so yielding context", this);
+            session.rollback();
             context.yield();
-            final String message = String.format("%s successfully connected to %s, but it has no flowfiles to provide, yielding", this, url);
-            logger.debug(message);
             return;
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessRemoteOutputPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessRemoteOutputPort.java
index 0543706..cd82fe7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessRemoteOutputPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessRemoteOutputPort.java
@@ -17,7 +17,6 @@
 package org.apache.nifi.stateless.core;
 
 import org.apache.nifi.events.EventReporter;
-import org.apache.nifi.remote.exception.NoValidPeerException;
 import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.DataUnit;
@@ -115,8 +114,8 @@ public class StatelessRemoteOutputPort extends AbstractStatelessComponent {
         try {
             final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
             if (transaction == null) {
-                getLogger().debug("No flowfiles to receive");
-                return true;
+                getLogger().error("Unable to create a transaction for Remote Process Group {} to pull from port {}", new Object[]{url, name});
+                return false;
             }
 
             final Queue<StatelessFlowFile> destinationQueue = new LinkedList<>();
@@ -140,9 +139,6 @@ public class StatelessRemoteOutputPort extends AbstractStatelessComponent {
 
             transaction.confirm();
             transaction.complete();
-        } catch (final NoValidPeerException e) {
-            getLogger().error("Unable to create a transaction for Remote Process Group {} to pull from port {}", new Object[]{url, name});
-            return false;
         } catch (final Exception e) {
             getLogger().error("Failed to receive FlowFile via site-to-site", e);
             return false;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
index f74d66e..c8787d3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
@@ -31,7 +31,6 @@ import org.apache.nifi.authorization.PublicPortAuthorizable;
 import org.apache.nifi.authorization.resource.ResourceType;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.remote.HttpRemoteSiteListener;
 import org.apache.nifi.remote.Peer;
 import org.apache.nifi.remote.PeerDescription;
@@ -78,7 +77,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.List;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 
@@ -92,7 +90,6 @@ 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 static org.apache.nifi.remote.protocol.http.HttpHeaders.PROTOCOL_VERSION;
 
 /**
  * RESTful endpoint for managing a SiteToSite connection.
@@ -208,21 +205,9 @@ public class DataTransferResource extends ApplicationResource {
         final int transportProtocolVersion = validationResult.transportProtocolVersion;
 
         try {
-            HttpFlowFileServerProtocol serverProtocol = initiateServerProtocol(req, peer, transportProtocolVersion);
-
-            int protocolVersion = Integer.parseUnsignedInt(req.getHeader(PROTOCOL_VERSION));
-
-            if ((protocolVersion >= 2) && PORT_TYPE_OUTPUT.equals(portType)) {
-                List<Connection> connectionList = serverProtocol.getPort().getIncomingConnections();
-                if (connectionList.stream().allMatch(c -> c.getFlowFileQueue().isEmpty())) {
-                    // Transaction could be created, but there is nothing to transfer. Just return 200.
-                    logger.debug("Output port has no flowfiles to transfer, returning 200");
-                    transactionManager.cancelTransaction(transactionId);
-                    return noCache(Response.status(Response.Status.NO_CONTENT)).type(MediaType.TEXT_PLAIN).entity("No flowfiles available").build();
-                }
-            }
-
             // Execute handshake.
+            initiateServerProtocol(req, peer, transportProtocolVersion);
+
             TransactionResultEntity entity = new TransactionResultEntity();
             entity.setResponseCode(ResponseCode.PROPERTIES_OK.getCode());
             entity.setMessage("Handshake properties are valid, and port is running. A transaction is created:" + transactionId);
diff --git a/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteCliMain.java b/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteCliMain.java
index 00bdaf8..e57dbbd 100644
--- a/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteCliMain.java
+++ b/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteCliMain.java
@@ -30,7 +30,6 @@ import org.apache.commons.cli.ParseException;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.KeystoreType;
 import org.apache.nifi.remote.client.SiteToSiteClient;
-import org.apache.nifi.remote.exception.NoContentException;
 import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 import org.apache.nifi.remote.protocol.http.HttpProxy;
 import org.apache.nifi.util.FormatUtils;
@@ -242,8 +241,6 @@ public class SiteToSiteCliMain {
                 } else {
                     new SiteToSiteReceiver(siteToSiteClient, output).receiveFiles();
                 }
-            } catch (final NoContentException e) {
-                System.out.println("Remote port has no flowfiles");
             }
         } catch (Exception e) {
             printUsage(e.getMessage(), options);
diff --git a/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteReceiver.java b/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteReceiver.java
index 82d19d2..88ad8f3 100644
--- a/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteReceiver.java
+++ b/nifi-toolkit/nifi-toolkit-s2s/src/main/java/org/apache/nifi/toolkit/s2s/SiteToSiteReceiver.java
@@ -22,7 +22,6 @@ import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransactionCompletion;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.SiteToSiteClient;
-import org.apache.nifi.remote.exception.NoContentException;
 import org.apache.nifi.remote.protocol.DataPacket;
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonGenerator;
@@ -46,9 +45,6 @@ public class SiteToSiteReceiver {
 
     public TransactionCompletion receiveFiles() throws IOException {
         Transaction transaction = siteToSiteClient.createTransaction(TransferDirection.RECEIVE);
-        if (transaction == null) {
-            throw new NoContentException("Remote side has no flowfiles to provide");
-        }
         JsonGenerator jsonGenerator = new JsonFactory().createJsonGenerator(output);
         jsonGenerator.writeStartArray();
         DataPacket dataPacket;


[nifi] 28/47: NIFI-7050 ConsumeJMS is not yielded in case of exception

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 4412460331ae3e751f4b699ec82f8bd292b9d220
Author: Gardella Juan Pablo <ju...@pitzil.com>
AuthorDate: Tue Jan 21 17:07:53 2020 -0300

    NIFI-7050 ConsumeJMS is not yielded in case of exception
    
    This closes #4004.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../org/apache/nifi/jms/processors/ConsumeJMS.java |  1 +
 .../apache/nifi/jms/processors/ConsumeJMSIT.java   | 25 ++++++++++++++++++++++
 2 files changed, 26 insertions(+)

diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index 80f9457..357e2f9 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -280,6 +280,7 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
             });
         } catch(Exception e) {
             consumer.setValid(false);
+            context.yield();
             throw e; // for backward compatibility with exception handling in flows
         }
     }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java
index 085db0e..714b950 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java
@@ -38,6 +38,7 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.MockProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
@@ -393,6 +394,30 @@ public class ConsumeJMSIT {
         }
     }
 
+    @Test(timeout = 10000)
+    public void whenExceptionIsRaisedTheProcessorShouldBeYielded() throws Exception {
+        TestRunner runner = TestRunners.newTestRunner(new ConsumeJMS());
+        JMSConnectionFactoryProviderDefinition cs = mock(JMSConnectionFactoryProviderDefinition.class);
+        ActiveMQConnectionFactory cf = new ActiveMQConnectionFactory("tcp://invalidhost:9999?soTimeout=3");
+
+        when(cs.getIdentifier()).thenReturn("cfProvider");
+        when(cs.getConnectionFactory()).thenReturn(cf);
+        runner.addControllerService("cfProvider", cs);
+        runner.enableControllerService(cs);
+
+        runner.setProperty(ConsumeJMS.CF_SERVICE, "cfProvider");
+        runner.setProperty(ConsumeJMS.DESTINATION, "foo");
+        runner.setProperty(ConsumeJMS.DESTINATION_TYPE, ConsumeJMS.TOPIC);
+
+        try {
+            runner.run();
+            fail("The test was implemented in a way this line should not be reached.");
+        } catch (AssertionError e) {
+        } finally {
+            assertTrue("In case of an exception, the processor should be yielded.", ((MockProcessContext) runner.getProcessContext()).isYieldCalled());
+        }
+    }
+
     private static void publishAMessage(ActiveMQConnectionFactory cf, final String destinationName, String messageContent) throws JMSException {
         // Publish a message.
         try (Connection conn = cf.createConnection();


[nifi] 34/47: NIFI-7241: When updating Process Group to match VersionedProcessGroup, remove any connections before recursing into child groups. This ensures that if a Port exists in child group A and is connected to a port in child group B, if the VersionedProcessGroup indicates to remove the port, that connection will be removed before attempting to remove the port. Updating and adding connections must still be done last, after all components have been added. But missing connections can be removed earl [...]

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit eeb7d698101a38731bee4d857cd24a4fb1bb5561
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Wed Mar 11 16:48:49 2020 -0400

    NIFI-7241: When updating Process Group to match VersionedProcessGroup, remove any connections before recursing into child groups. This ensures that if a Port exists in child group A and is connected to a port in child group B, if the VersionedProcessGroup indicates to remove the port, that connection will be removed before attempting to remove the port. Updating and adding connections must still be done last, after all components have been added. But missing connections can be removed [...]
    joewitt Added Mark Payne provided update since this change was dependent on NIFI-6873 so it would work in nifi 1.11.4
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4136.
---
 .../apache/nifi/groups/StandardProcessGroup.java   | 40 +++++++++++++---------
 1 file changed, 23 insertions(+), 17 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 7cdb5b7..8364d95 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -3803,7 +3803,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             ControllerServiceNode service = servicesByVersionedId.get(proposedService.getIdentifier());
             if (service == null) {
                 service = addControllerService(group, proposedService, componentIdSeed);
-                LOG.info("Added {} to {}", service, this);
+                LOG.info("Added {} to {}", service, group);
                 servicesAdded.put(proposedService.getIdentifier(), service);
             }
 
@@ -3835,6 +3835,27 @@ public final class StandardProcessGroup implements ProcessGroup {
             controllerServicesRemoved.remove(proposedService.getIdentifier());
         }
 
+        // Before we can update child groups, we must first remove any connections that are connected to those child groups' input/output ports.
+        // We cannot add or update connections yet, though. That must be done at the end, as it's possible that the component that is the source/destination of the connection
+        // has not yet been added.
+        final Map<String, Connection> connectionsByVersionedId = group.getConnections().stream()
+                .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(component.getIdentifier()), Function.identity()));
+        final Set<String> connectionsRemoved = new HashSet<>(connectionsByVersionedId.keySet());
+
+        for (final VersionedConnection proposedConnection : proposed.getConnections()) {
+            connectionsRemoved.remove(proposedConnection.getIdentifier());
+        }
+
+        // Connections must be the first thing to remove, not the last. Otherwise, we will fail
+        // to remove a component if it has a connection going to it!
+        for (final String removedVersionedId : connectionsRemoved) {
+            final Connection connection = connectionsByVersionedId.get(removedVersionedId);
+            LOG.info("Removing {} from {}", connection, group);
+            group.removeConnection(connection);
+            flowManager.onConnectionRemoved(connection);
+        }
+
+
         // Child groups
         final Map<String, ProcessGroup> childGroupsByVersionedId = group.getProcessGroups().stream()
                 .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(component.getIdentifier()), Function.identity()));
@@ -4019,11 +4040,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         }
 
 
-        // Connections
-        final Map<String, Connection> connectionsByVersionedId = group.getConnections().stream()
-                .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(component.getIdentifier()), Function.identity()));
-        final Set<String> connectionsRemoved = new HashSet<>(connectionsByVersionedId.keySet());
-
+        // Add and update Connections
         for (final VersionedConnection proposedConnection : proposed.getConnections()) {
             final Connection connection = connectionsByVersionedId.get(proposedConnection.getIdentifier());
             if (connection == null) {
@@ -4037,21 +4054,10 @@ public final class StandardProcessGroup implements ProcessGroup {
                 updateConnection(connection, proposedConnection);
                 LOG.info("Updated {}", connection);
             }
-
-            connectionsRemoved.remove(proposedConnection.getIdentifier());
         }
 
         // Remove components that exist in the local flow but not the remote flow.
 
-        // Connections must be the first thing to remove, not the last. Otherwise, we will fail
-        // to remove a component if it has a connection going to it!
-        for (final String removedVersionedId : connectionsRemoved) {
-            final Connection connection = connectionsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", connection, group);
-            group.removeConnection(connection);
-            flowManager.onConnectionRemoved(connection);
-        }
-
         // Once the appropriate connections have been removed, we may now update Processors' auto-terminated relationships.
         // We cannot do this above, in the 'updateProcessor' call because if a connection is removed and changed to auto-terminated,
         // then updating this in the updateProcessor call above would attempt to set the Relationship to being auto-terminated while a


[nifi] 40/47: NIFI-7249: Force String keys in maps in DataTypeUtils.inferDataType()

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 76c5335cb54f231a2cc31df65d377077ef85ab31
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Thu Mar 12 14:46:58 2020 -0400

    NIFI-7249: Force String keys in maps in DataTypeUtils.inferDataType()
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4139.
---
 .../serialization/record/util/DataTypeUtils.java   | 18 +++++++-
 .../serialization/record/TestDataTypeUtils.java    | 31 +++++++++++++
 .../org/apache/nifi/avro/TestAvroTypeUtil.java     | 51 ++++++++++++++++++++++
 3 files changed, 99 insertions(+), 1 deletion(-)

diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
index b1daa91..009332e 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
@@ -474,7 +474,23 @@ public class DataTypeUtils {
 
         // A value of a Map could be either a Record or a Map type. In either case, it must have Strings as keys.
         if (value instanceof Map) {
-            final Map<String, ?> map = (Map<String, ?>) value;
+            final Map<String, Object> map;
+            // Only transform the map if the keys aren't strings
+            boolean allStrings = true;
+            for (final Object key : ((Map<?, ?>) value).keySet()) {
+                if (!(key instanceof String)) {
+                    allStrings = false;
+                    break;
+                }
+            }
+
+            if (allStrings) {
+                map = (Map<String, Object>) value;
+            } else {
+                final Map<?, ?> m = (Map<?, ?>) value;
+                map = new HashMap<>(m.size());
+                m.forEach((k, v) -> map.put(k == null ? null : k.toString(), v));
+            }
             return inferRecordDataType(map);
 //            // Check if all types are the same.
 //            if (map.isEmpty()) {
diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java
index 30b2a60..8152d4f 100644
--- a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java
+++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java
@@ -19,6 +19,7 @@ package org.apache.nifi.serialization.record;
 
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.type.ChoiceDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
 import org.junit.Test;
@@ -443,6 +444,36 @@ public class TestDataTypeUtils {
     }
 
     @Test
+    public void testInferTypeWithMapStringKeys() {
+        Map<String, String> map = new HashMap<>();
+        map.put("a", "Hello");
+        map.put("b", "World");
+
+        RecordDataType expected = (RecordDataType)RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(Arrays.asList(
+                new RecordField("a", RecordFieldType.STRING.getDataType()),
+                new RecordField("b", RecordFieldType.STRING.getDataType())
+        )));
+
+        DataType actual = DataTypeUtils.inferDataType(map, null);
+        assertEquals(expected, actual);
+    }
+
+    @Test
+    public void testInferTypeWithMapNonStringKeys() {
+        Map<Integer, String> map = new HashMap<>();
+        map.put(1, "Hello");
+        map.put(2, "World");
+
+        RecordDataType expected = (RecordDataType)RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(Arrays.asList(
+                new RecordField("1", RecordFieldType.STRING.getDataType()),
+                new RecordField("2", RecordFieldType.STRING.getDataType())
+        )));
+
+        DataType actual = DataTypeUtils.inferDataType(map, null);
+        assertEquals(expected, actual);
+    }
+
+    @Test
     public void testFindMostSuitableTypeWithBoolean() {
         testFindMostSuitableType(true, RecordFieldType.BOOLEAN.getDataType());
     }
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
index a89ebe4..5b5f55dd 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
@@ -29,6 +29,7 @@ import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.avro.util.Utf8;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
@@ -640,4 +641,54 @@ public class TestAvroTypeUtil {
         // THEN
         assertEquals(expected, actualAfterReverse);
     }
+
+    @Test
+    public void testConvertAvroMap() {
+        // GIVEN
+        Map<?, ?> expected = new HashMap<String, Object>() {{
+            put(
+                    "nullableMapField",
+                    new HashMap<String, Object>() {{
+                        put("key1", "value1");
+                        put("key2", "value2");
+                    }}
+            );
+        }};
+
+        Schema nullableMapFieldAvroSchema = Schema.createUnion(
+                Schema.create(Type.NULL),
+                Schema.create(Type.INT),
+                Schema.createMap(Schema.create(Type.STRING))
+        );
+
+        Schema avroRecordSchema = Schema.createRecord(
+                "record", "doc", "namespace", false,
+                Arrays.asList(
+                        new Field("nullableMapField", nullableMapFieldAvroSchema, "nullable map field", (Object)null)
+                )
+        );
+
+        Map<?, ?> value = new HashMap<Utf8, Object>(){{
+            put(new Utf8("key1"), "value1");
+            put(new Utf8("key2"), "value2");
+        }};
+
+        Record avroRecord = new GenericRecordBuilder(avroRecordSchema)
+                .set("nullableMapField", value)
+                .build();
+
+        RecordSchema nifiRecordSchema = new SimpleRecordSchema(
+                Arrays.asList(
+                        new RecordField("nullableMapField", RecordFieldType.CHOICE.getChoiceDataType(
+                                RecordFieldType.MAP.getMapDataType(RecordFieldType.STRING.getDataType())
+                        ))
+                )
+        );
+
+        // WHEN
+        Object actual = AvroTypeUtil.convertAvroRecordToMap(avroRecord, nifiRecordSchema);
+
+        // THEN
+        assertEquals(expected, actual);
+    }
 }


[nifi] 32/47: NIFI-7242: When a Parameter is changed, any property referencing that parameter should have its #onPropertyModified method called. Also renamed Accumulo tests to integration tests because they start embedded servers and connect to them, which caused failures in my environment. Also fixed a bug in TestLengthDelimitedJournal because it was resulting in failures when building locally as well.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6a3f6f6ceaa2a9a7b29880bf353075428c128623
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Wed Mar 11 09:41:56 2020 -0400

    NIFI-7242: When a Parameter is changed, any property referencing that parameter should have its #onPropertyModified method called. Also renamed Accumulo tests to integration tests because they start embedded servers and connect to them, which caused failures in my environment. Also fixed a bug in TestLengthDelimitedJournal because it was resulting in failures when building locally as well.
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4134.
---
 .../attribute/expression/language/TestQuery.java   |   5 +
 .../language/TestStandardPreparedQuery.java        |   5 +
 .../org/apache/nifi/parameter/ParameterLookup.java |  14 +++
 .../nifi/wali/TestLengthDelimitedJournal.java      |  13 ++-
 .../{TestPutRecord.java => PutRecordIT.java}       |   2 +-
 .../{TestScanAccumulo.java => ScanAccumuloIT.java} |   2 +-
 .../nifi/controller/AbstractComponentNode.java     | 103 ++++++++++++++++-
 .../org/apache/nifi/controller/ComponentNode.java  |   7 ++
 .../nifi/controller/PropertyConfiguration.java     |  26 ++---
 .../java/org/apache/nifi/groups/ProcessGroup.java  |   5 +-
 .../apache/nifi/parameter/ParameterContext.java    |   9 +-
 .../org/apache/nifi/parameter/ParameterUpdate.java |  35 +++---
 .../nifi/controller/TestAbstractComponentNode.java | 123 ++++++++++++++++++++-
 .../apache/nifi/groups/StandardProcessGroup.java   |  88 ++++++++++++---
 .../nifi/parameter/StandardParameterContext.java   |  23 +++-
 .../nifi/parameter/StandardParameterUpdate.java    |  83 ++++++++++++++
 .../controller/service/mock/MockProcessGroup.java  |   3 +-
 17 files changed, 473 insertions(+), 73 deletions(-)

diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
index d48129d..7f17f12 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
@@ -2228,5 +2228,10 @@ public class TestQuery {
         public boolean isEmpty() {
             return parameters.isEmpty();
         }
+
+        @Override
+        public long getVersion() {
+            return 0;
+        }
     }
 }
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
index 09c50d9..546160d 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
@@ -253,6 +253,11 @@ public class TestStandardPreparedQuery {
             }
 
             @Override
+            public long getVersion() {
+                return 0;
+            }
+
+            @Override
             public boolean isEmpty() {
                 return parameters.isEmpty();
             }
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
index 3d97170..1ec6790 100644
--- a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
+++ b/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
@@ -33,6 +33,15 @@ public interface ParameterLookup {
      */
     boolean isEmpty();
 
+    /**
+     * Indicates the current Version of the Parameter Context. Each time that the Parameter Context is updated, its version is incremented. This allows
+     * other components to know whether or not the values have changed since some other point in time. The version may or may not be persisted across
+     * restarts of the application.
+     *
+     * @return the current version
+     */
+    long getVersion();
+
 
     ParameterLookup EMPTY = new ParameterLookup() {
         @Override
@@ -41,6 +50,11 @@ public interface ParameterLookup {
         }
 
         @Override
+        public long getVersion() {
+            return 0;
+        }
+
+        @Override
         public boolean isEmpty() {
             return true;
         }
diff --git a/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestLengthDelimitedJournal.java b/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestLengthDelimitedJournal.java
index d54ba9c..c3572f9 100644
--- a/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestLengthDelimitedJournal.java
+++ b/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestLengthDelimitedJournal.java
@@ -43,6 +43,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
@@ -480,12 +481,12 @@ public class TestLengthDelimitedJournal {
         // the BADOS will sleep for 1 second before writing. This allwos other threads to trigger corruption in the repo in the meantime.
         final ByteArrayDataOutputStream pausingBados = new ByteArrayDataOutputStream(4096) {
             private final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            private int count = 0;
+            private final AtomicInteger count = new AtomicInteger(0);
 
             @Override
             public ByteArrayOutputStream getByteArrayOutputStream() {
                 // Pause only on the second iteration.
-                if (count++ == 1) {
+                if (count.getAndIncrement() == 1) {
                     try {
                         Thread.sleep(1000L);
                     } catch (final InterruptedException ie) {
@@ -503,11 +504,11 @@ public class TestLengthDelimitedJournal {
 
 
         final Supplier<ByteArrayDataOutputStream> badosSupplier = new Supplier<ByteArrayDataOutputStream>() {
-            private int count = 0;
+            private final AtomicInteger count = new AtomicInteger(0);
 
             @Override
             public ByteArrayDataOutputStream get() {
-                if (count++ == 0) {
+                if (count.getAndIncrement() == 0) {
                     return pausingBados;
                 }
 
@@ -525,11 +526,11 @@ public class TestLengthDelimitedJournal {
         final Thread[] threads = new Thread[2];
 
         final LengthDelimitedJournal<DummyRecord> journal = new LengthDelimitedJournal<DummyRecord>(journalFile, serdeFactory, corruptingStreamPool, 0L) {
-            private int count = 0;
+            private final AtomicInteger count = new AtomicInteger(0);
 
             @Override
             protected void poison(final Throwable t)  {
-                if (count++ == 0) { // it is only important that we sleep the first time. If we sleep every time, it just slows the test down.
+                if (count.getAndIncrement() == 0) { // it is only important that we sleep the first time. If we sleep every time, it just slows the test down.
                     try {
                         Thread.sleep(3000L);
                     } catch (InterruptedException e) {
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/TestPutRecord.java b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/PutRecordIT.java
similarity index 99%
rename from nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/TestPutRecord.java
rename to nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/PutRecordIT.java
index 2d45a48..c8bf47c 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/TestPutRecord.java
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/PutRecordIT.java
@@ -52,7 +52,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 
-public class TestPutRecord {
+public class PutRecordIT {
 
     public static final String DEFAULT_COLUMN_FAMILY = "family1";
 
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/TestScanAccumulo.java b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/ScanAccumuloIT.java
similarity index 99%
rename from nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/TestScanAccumulo.java
rename to nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/ScanAccumuloIT.java
index 3be8c72..4abd8cb 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/TestScanAccumulo.java
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/test/java/org/apache/nifi/accumulo/processors/ScanAccumuloIT.java
@@ -54,7 +54,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 
-public class TestScanAccumulo {
+public class ScanAccumuloIT {
 
     public static final String DEFAULT_COLUMN_FAMILY = "family1";
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
index adf410f..a1f5af1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
@@ -36,9 +36,12 @@ import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
 import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.parameter.ParameterParser;
 import org.apache.nifi.parameter.ParameterReference;
 import org.apache.nifi.parameter.ParameterTokenList;
+import org.apache.nifi.parameter.ParameterUpdate;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.util.CharacterFilterUtils;
 import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
@@ -815,12 +818,110 @@ public abstract class AbstractComponentNode implements ComponentNode {
     }
 
 
-    private void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+    protected void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) {
             getComponent().onPropertyModified(descriptor, oldValue, newValue);
         }
     }
 
+    @Override
+    public void onParametersModified(final Map<String, ParameterUpdate> updatedParameters) {
+        // If the component doesn't reference any parameters, then there's nothing to be done.
+        if (!isReferencingParameter()) {
+            return;
+        }
+
+        final ParameterLookup previousParameterLookup = createParameterLookupForPreviousValues(updatedParameters);
+
+        // For any Property that references an updated Parameter, we need to call onPropertyModified().
+        // Additionally, we need to trigger validation to run if this component is affected by the parameter update.
+        boolean componentAffected = false;
+        for (final Map.Entry<PropertyDescriptor, PropertyConfiguration> entry : properties.entrySet()) {
+            final PropertyDescriptor propertyDescriptor = entry.getKey();
+            final PropertyConfiguration configuration = entry.getValue();
+
+            // Determine if this property is affected by the Parameter Update
+            boolean propertyAffected = false;
+            final List<ParameterReference> parameterReferences = configuration.getParameterReferences();
+            for (final ParameterReference reference : parameterReferences) {
+                final String referencedParamName = reference.getParameterName();
+                if (updatedParameters.containsKey(referencedParamName)) {
+                    propertyAffected = true;
+                    componentAffected = true;
+                    break;
+                }
+            }
+
+            if (propertyAffected) {
+                final String previousValue = configuration.getEffectiveValue(previousParameterLookup);
+                final String updatedValue = configuration.getEffectiveValue(getParameterContext());
+
+                // Check if the value of the property is truly affected. It's possible that we could have a property configured as something like "#{a}#{b}"
+                // Where parameter a = "abc-" and b = "cba". The update could change a to "abc" and b to "-cba". As a result, the property value previously was "abc-cba" and still is.
+                // In such a case, we should not call onPropertyModified.
+                final boolean propertyUpdated = !Objects.equals(previousValue, updatedValue);
+                if (propertyUpdated) {
+                    try {
+                        logger.debug("Parameter Context updated, resulting in property {} of {} changing. Calling onPropertyModified().", propertyDescriptor, this);
+                        onPropertyModified(propertyDescriptor, previousValue, updatedValue);
+                    } catch (final Exception e) {
+                        // nothing really to do here...
+                        logger.error("Failed to notify {} that property {} changed", this, propertyDescriptor, e);
+                    }
+                } else {
+                    logger.debug("Parameter Context updated, and property {} of {} does reference the updated Parameters. However, the overall property value remained unchanged so will not call " +
+                        "onPropertyModified().", propertyDescriptor, this);
+                }
+            }
+        }
+
+        // If this component is affected by the Parameter change, we need to re-validate
+        if (componentAffected) {
+            logger.debug("Configuration of {} changed due to an update to Parameter Context. Resetting validation state", this);
+            resetValidationState();
+        }
+    }
+
+    private ParameterLookup createParameterLookupForPreviousValues(final Map<String, ParameterUpdate> updatedParameters) {
+        final ParameterContext currentContext = getParameterContext();
+        return new ParameterLookup() {
+            @Override
+            public Optional<Parameter> getParameter(final String parameterName) {
+                final Optional<Parameter> optionalParameter = currentContext == null ? Optional.empty() : currentContext.getParameter(parameterName);
+
+                // Check if there's an update to the parameter. If not, just return the parameter as-is.
+                final ParameterUpdate parameterUpdate = updatedParameters.get(parameterName);
+                if (parameterUpdate == null) {
+                    return optionalParameter;
+                }
+
+                // There is an update to the parameter. We want to return the previous value of the Parameter.
+                final ParameterDescriptor parameterDescriptor;
+                if (optionalParameter.isPresent()) {
+                    parameterDescriptor = optionalParameter.get().getDescriptor();
+                } else {
+                    parameterDescriptor = new ParameterDescriptor.Builder()
+                        .name(parameterName)
+                        .description("")
+                        .sensitive(true)
+                        .build();
+                }
+
+                final Parameter updatedParameter = new Parameter(parameterDescriptor, parameterUpdate.getPreviousValue());
+                return Optional.of(updatedParameter);
+            }
+
+            @Override
+            public boolean isEmpty() {
+                return (currentContext == null || currentContext.isEmpty()) && updatedParameters.isEmpty();
+            }
+
+            @Override
+            public long getVersion() {
+                return 0;
+            }
+        };
+    }
 
     @Override
     public ValidationStatus getValidationStatus() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java
index c8b779c..de7612a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java
@@ -34,6 +34,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.validation.ValidationState;
 import org.apache.nifi.components.validation.ValidationStatus;
 import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterUpdate;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 
 import java.net.URL;
@@ -72,6 +73,12 @@ public interface ComponentNode extends ComponentAuthorizable {
     boolean isReferencingParameter();
 
     /**
+     * Notifies the Component that the value of a parameter has changed
+     * @param parameterUpdates a Map of Parameter name to a ParameterUpdate that describes how the Parameter changed
+     */
+    void onParametersModified(Map<String, ParameterUpdate> parameterUpdates);
+
+    /**
      * <p>
      * Pause triggering asynchronous validation to occur when the component is updated. Often times, it is necessary
      * to update several aspects of a component, such as the properties and annotation data, at once. When this occurs,
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java
index 9121d02..d25a981 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/PropertyConfiguration.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.controller;
 
-import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.parameter.ParameterReference;
 import org.apache.nifi.parameter.ParameterTokenList;
 import org.apache.nifi.parameter.StandardParameterTokenList;
@@ -44,7 +44,7 @@ public class PropertyConfiguration {
         return rawValue;
     }
 
-    public String getEffectiveValue(final ParameterContext parameterContext) {
+    public String getEffectiveValue(final ParameterLookup parameterLookup) {
         if (rawValue == null) {
             return null;
         }
@@ -57,12 +57,12 @@ public class PropertyConfiguration {
         // cache the Effective Value because we may have a different Parameter Context. So, we cache a Tuple of
         // the Parameter Context and the effective value for that Parameter Context.
         final ComputedEffectiveValue computedEffectiveValue = effectiveValue.get();
-        if (computedEffectiveValue != null && computedEffectiveValue.matches(parameterContext)) {
+        if (computedEffectiveValue != null && computedEffectiveValue.matches(parameterLookup)) {
             return computedEffectiveValue.getValue();
         }
 
-        final String substituted = parameterTokenList.substitute(parameterContext);
-        final ComputedEffectiveValue updatedValue = new ComputedEffectiveValue(parameterContext, substituted);
+        final String substituted = parameterTokenList.substitute(parameterLookup);
+        final ComputedEffectiveValue updatedValue = new ComputedEffectiveValue(parameterLookup, substituted);
         effectiveValue.compareAndSet(computedEffectiveValue, updatedValue);
         return substituted;
     }
@@ -96,13 +96,13 @@ public class PropertyConfiguration {
 
 
     public static class ComputedEffectiveValue {
-        private final ParameterContext parameterContext;
+        private final ParameterLookup parameterLookup;
         private final long contextVersion;
         private final String value;
 
-        public ComputedEffectiveValue(final ParameterContext parameterContext, final String value) {
-            this.parameterContext = parameterContext;
-            this.contextVersion = parameterContext == null ? -1 : parameterContext.getVersion();
+        public ComputedEffectiveValue(final ParameterLookup parameterLookup, final String value) {
+            this.parameterLookup = parameterLookup;
+            this.contextVersion = parameterLookup == null ? -1 : parameterLookup.getVersion();
             this.value = value;
         }
 
@@ -110,16 +110,16 @@ public class PropertyConfiguration {
             return value;
         }
 
-        public boolean matches(final ParameterContext context) {
-            if (!Objects.equals(context, this.parameterContext)) {
+        public boolean matches(final ParameterLookup parameterLookup) {
+            if (!Objects.equals(parameterLookup, this.parameterLookup)) {
                 return false;
             }
 
-            if (context == null) {
+            if (parameterLookup == null) {
                 return true;
             }
 
-            return context.getVersion() == contextVersion;
+            return parameterLookup.getVersion() == contextVersion;
         }
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
index 1c628bd..8e5205e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
@@ -34,6 +34,7 @@ import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterUpdate;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
@@ -1035,6 +1036,8 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
 
     /**
      * Called to notify the Process Group whenever the Parameter Context that it is bound to has changed.
+     *
+     * @param updatedParameters a Map of parameter name to the ParameterUpdate that describes how the Parameter was updated
      */
-    void onParameterContextUpdated();
+    void onParameterContextUpdated(Map<String, ParameterUpdate> updatedParameters);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterContext.java
index f061320..9bfc63b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterContext.java
@@ -91,12 +91,5 @@ public interface ParameterContext extends ParameterLookup, ComponentAuthorizable
      */
     ParameterReferenceManager getParameterReferenceManager();
 
-    /**
-     * Indicates the current Version of the Parameter Context. Each time that the Parameter Context is updated, its version is incremented. This allows
-     * other components to know whether or not the values have changed since some other point in time. The version may or may not be persisted across
-     * restarts of the application.
-     *
-     * @return the current version
-     */
-    long getVersion();
+
 }
diff --git a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterUpdate.java
similarity index 53%
copy from nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterUpdate.java
index 3d97170..a5743f3 100644
--- a/nifi-commons/nifi-parameter/src/main/java/org/apache/nifi/parameter/ParameterLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/parameter/ParameterUpdate.java
@@ -16,33 +16,24 @@
  */
 package org.apache.nifi.parameter;
 
-import java.util.Optional;
-
-public interface ParameterLookup {
-
+public interface ParameterUpdate {
     /**
-     * Returns the Parameter with the given name
-     * @param parameterName the name of the Parameter
-     * @return the Parameter with the given name or an empty Optional if no Parameter exists with that name
+     * @return the name of the Parameter
      */
-    Optional<Parameter> getParameter(String parameterName);
+    String getParameterName();
 
     /**
-     * Returns false if any Parameters are available, true if no Parameters have been defined
-     * @return true if empty
+     * @return the previous value of the Parameter, or <code>null</code> if the Parameter is being added or previously had no value
      */
-    boolean isEmpty();
+    String getPreviousValue();
 
+    /**
+     * @return the updated value of the Parameter, or <code>null</code> if the Parameter was removed or previously had no value
+     */
+    String getUpdatedValue();
 
-    ParameterLookup EMPTY = new ParameterLookup() {
-        @Override
-        public Optional<Parameter> getParameter(final String parameterName) {
-            return Optional.empty();
-        }
-
-        @Override
-        public boolean isEmpty() {
-            return true;
-        }
-    };
+    /**
+     * @return <code>true</code> if the Parameter is sensitive, <code>false</code> otherwise.
+     */
+    boolean isSensitive();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
index 65a2471..6ea143e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
@@ -17,25 +17,33 @@
 
 package org.apache.nifi.controller;
 
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.validation.ValidationStatus;
 import org.apache.nifi.components.validation.ValidationTrigger;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.apache.nifi.parameter.ParameterUpdate;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 import java.net.URL;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -51,6 +59,52 @@ public class TestAbstractComponentNode {
         assertEquals(ValidationStatus.VALIDATING, status);
     }
 
+    public void testOnParametersModified() {
+        final AtomicLong validationCount = new AtomicLong(0L);
+        final ValidationTrigger validationTrigger = new ValidationTrigger() {
+            @Override
+            public void triggerAsync(ComponentNode component) {
+                validationCount.incrementAndGet();
+            }
+
+            @Override
+            public void trigger(ComponentNode component) {
+                validationCount.incrementAndGet();
+            }
+        };
+
+        final List<PropertyModification> propertyModifications = new ArrayList<>();
+        final ValidationControlledAbstractComponentNode node = new ValidationControlledAbstractComponentNode(0, validationTrigger) {
+            @Override
+            protected void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+                propertyModifications.add(new PropertyModification(descriptor, oldValue, newValue));
+                super.onPropertyModified(descriptor, oldValue, newValue);
+            }
+        };
+
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("abc", "#{abc}");
+        node.setProperties(properties);
+
+        final ParameterContext context = Mockito.mock(ParameterContext.class);
+        final ParameterDescriptor paramDescriptor = new ParameterDescriptor.Builder()
+            .name("abc")
+            .description("")
+            .sensitive(false)
+            .build();
+        final Parameter param = new Parameter(paramDescriptor, "123");
+        Mockito.doReturn(param).when(context).getParameter("abc");
+
+        final Map<String, ParameterUpdate> updatedParameters = new HashMap<>();
+        updatedParameters.put("abc", new MockParameterUpdate("abc", "xyz", "123", false));
+        node.onParametersModified(updatedParameters);
+
+        assertEquals(1, propertyModifications.size());
+        final PropertyModification mod = propertyModifications.get(0);
+        assertEquals("xyz", mod.getPreviousValue());
+        assertEquals("123", mod.getUpdatedValue());
+    }
+
     @Test(timeout = 10000)
     public void testValidationTriggerPaused() throws InterruptedException {
         final AtomicLong validationCount = new AtomicLong(0L);
@@ -84,6 +138,7 @@ public class TestAbstractComponentNode {
 
     private static class ValidationControlledAbstractComponentNode extends AbstractComponentNode {
         private final long pauseMillis;
+        private volatile ParameterContext paramContext = null;
 
         public ValidationControlledAbstractComponentNode(final long pauseMillis, final ValidationTrigger validationTrigger) {
             super("id", Mockito.mock(ValidationContextFactory.class), Mockito.mock(ControllerServiceProvider.class), "unit test component",
@@ -168,7 +223,69 @@ public class TestAbstractComponentNode {
 
         @Override
         protected ParameterContext getParameterContext() {
-            return null;
+            return paramContext;
+        }
+
+        protected void setParameterContext(final ParameterContext parameterContext) {
+            this.paramContext = parameterContext;
+        }
+    }
+
+    private static class PropertyModification {
+        private final PropertyDescriptor propertyDescriptor;
+        private final String previousValue;
+        private final String updatedValue;
+
+        public PropertyModification(final PropertyDescriptor propertyDescriptor, final String previousValue, final String updatedValue) {
+            this.propertyDescriptor = propertyDescriptor;
+            this.previousValue = previousValue;
+            this.updatedValue = updatedValue;
+        }
+
+        public PropertyDescriptor getPropertyDescriptor() {
+            return propertyDescriptor;
+        }
+
+        public String getPreviousValue() {
+            return previousValue;
+        }
+
+        public String getUpdatedValue() {
+            return updatedValue;
+        }
+    }
+
+    private static class MockParameterUpdate implements ParameterUpdate {
+        private final String parameterName;
+        private final String oldValue;
+        private final String newValue;
+        private final boolean sensitive;
+
+        public MockParameterUpdate(final String parameterName, final String oldValue, final String newValue, final boolean sensitive) {
+            this.parameterName = parameterName;
+            this.oldValue = oldValue;
+            this.newValue = newValue;
+            this.sensitive = sensitive;
+        }
+
+        @Override
+        public String getParameterName() {
+            return parameterName;
+        }
+
+        @Override
+        public String getPreviousValue() {
+            return oldValue;
+        }
+
+        @Override
+        public String getUpdatedValue() {
+            return newValue;
+        }
+
+        @Override
+        public boolean isSensitive() {
+            return sensitive;
         }
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index dedb6a9..7cdb5b7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -75,6 +75,8 @@ import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterDescriptor;
 import org.apache.nifi.parameter.ParameterReference;
+import org.apache.nifi.parameter.ParameterUpdate;
+import org.apache.nifi.parameter.StandardParameterUpdate;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.StandardProcessContext;
 import org.apache.nifi.registry.ComponentVariableRegistry;
@@ -156,6 +158,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
@@ -2911,32 +2914,91 @@ public final class StandardProcessGroup implements ProcessGroup {
     @Override
     public void setParameterContext(final ParameterContext parameterContext) {
         verifyCanSetParameterContext(parameterContext);
+
+        // Determine which parameters have changed so that components can be appropriately updated.
+        final Map<String, ParameterUpdate> updatedParameters = mapParameterUpdates(this.parameterContext, parameterContext);
+        LOG.debug("Parameter Context for {} changed from {} to {}. This resulted in {} Parameter Updates ({}). Notifying Processors/Controller Services of the updates.",
+            this, this.parameterContext, parameterContext, updatedParameters.size(), updatedParameters);
+
         this.parameterContext = parameterContext;
 
-        getProcessors().forEach(ProcessorNode::resetValidationState);
-        getControllerServices(false).forEach(ControllerServiceNode::resetValidationState);
+        if (!updatedParameters.isEmpty()) {
+            // Notify components that parameters have been updated
+            onParameterContextUpdated(updatedParameters);
+        }
     }
 
     @Override
-    public void onParameterContextUpdated() {
+    public void onParameterContextUpdated(final Map<String, ParameterUpdate> updatedParameters) {
         readLock.lock();
         try {
-            for (final ProcessorNode processorNode : getProcessors()) {
-                if (processorNode.isReferencingParameter() && processorNode.getScheduledState() != ScheduledState.RUNNING) {
-                    processorNode.resetValidationState();
-                }
+            getProcessors().forEach(proc -> proc.onParametersModified(updatedParameters));
+            getControllerServices(false).forEach(cs -> cs.onParametersModified(updatedParameters));
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    private Map<String, ParameterUpdate> mapParameterUpdates(final ParameterContext previousParameterContext, final ParameterContext updatedParameterContext) {
+        if (previousParameterContext == null && updatedParameterContext == null) {
+            return Collections.emptyMap();
+        }
+        if (updatedParameterContext == null) {
+            return createParameterUpdates(previousParameterContext, (descriptor, value) -> new StandardParameterUpdate(descriptor.getName(), value, null, descriptor.isSensitive()));
+        }
+        if (previousParameterContext == null) {
+            return createParameterUpdates(updatedParameterContext, (descriptor, value) -> new StandardParameterUpdate(descriptor.getName(), null, value, descriptor.isSensitive()));
+        }
+
+        // For each Parameter in the updated parameter context, add a ParameterUpdate to our map
+        final Map<String, ParameterUpdate> updatedParameters = new HashMap<>();
+        for (final Map.Entry<ParameterDescriptor, Parameter> entry : updatedParameterContext.getParameters().entrySet()) {
+            final ParameterDescriptor updatedDescriptor = entry.getKey();
+            final Parameter updatedParameter = entry.getValue();
+
+            final Optional<Parameter> previousParameterOption = previousParameterContext.getParameter(updatedDescriptor);
+            final String previousValue = previousParameterOption.map(Parameter::getValue).orElse(null);
+            final String updatedValue = updatedParameter.getValue();
+
+            if (!Objects.equals(previousValue, updatedValue)) {
+                final ParameterUpdate parameterUpdate = new StandardParameterUpdate(updatedDescriptor.getName(), previousValue, updatedValue, updatedDescriptor.isSensitive());
+                updatedParameters.put(updatedDescriptor.getName(), parameterUpdate);
             }
+        }
 
-            for (final ControllerServiceNode serviceNode : getControllerServices(false)) {
-                if (serviceNode.isReferencingParameter() && serviceNode.getState() == ControllerServiceState.DISABLING || serviceNode.getState() == ControllerServiceState.DISABLED) {
-                    serviceNode.resetValidationState();
-                }
+        // For each Parameter that was in the previous parameter context that is not in the updated Paramter Context, add a ParameterUpdate to our map with `null` for the updated value
+        for (final Map.Entry<ParameterDescriptor, Parameter> entry : previousParameterContext.getParameters().entrySet()) {
+            final ParameterDescriptor previousDescriptor = entry.getKey();
+            final Parameter previousParameter = entry.getValue();
+
+            final Optional<Parameter> updatedParameterOption = updatedParameterContext.getParameter(previousDescriptor);
+            if (updatedParameterOption.isPresent()) {
+                // The value exists in both Parameter Contexts. If it was changed, a Parameter Update has already been added to the map, above.
+                continue;
             }
-        } finally {
-            readLock.unlock();
+
+            final ParameterUpdate parameterUpdate = new StandardParameterUpdate(previousDescriptor.getName(), previousParameter.getValue(), null, previousDescriptor.isSensitive());
+            updatedParameters.put(previousDescriptor.getName(), parameterUpdate);
         }
+
+        return updatedParameters;
     }
 
+    private Map<String, ParameterUpdate> createParameterUpdates(final ParameterContext parameterContext, final BiFunction<ParameterDescriptor, String, ParameterUpdate> parameterUpdateMapper) {
+        final Map<String, ParameterUpdate> updatedParameters = new HashMap<>();
+
+        for (final Map.Entry<ParameterDescriptor, Parameter> entry : parameterContext.getParameters().entrySet()) {
+            final ParameterDescriptor parameterDescriptor = entry.getKey();
+            final Parameter parameter = entry.getValue();
+
+            final ParameterUpdate parameterUpdate = parameterUpdateMapper.apply(parameterDescriptor, parameter.getValue());
+            updatedParameters.put(parameterDescriptor.getName(), parameterUpdate);
+        }
+
+        return updatedParameters;
+    }
+
+
     @Override
     public void verifyCanSetParameterContext(final ParameterContext parameterContext) {
         readLock.lock();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java
index 7af68fc..b6cae26 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java
@@ -30,6 +30,7 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Objects;
@@ -107,33 +108,44 @@ public class StandardParameterContext implements ParameterContext {
             this.version++;
             verifyCanSetParameters(updatedParameters);
 
+            final Map<String, ParameterUpdate> parameterUpdates = new HashMap<>();
             boolean changeAffectingComponents = false;
             for (final Map.Entry<String, Parameter> entry : updatedParameters.entrySet()) {
                 final String parameterName = entry.getKey();
                 final Parameter parameter = entry.getValue();
 
                 if (parameter == null) {
-                    final ParameterDescriptor parameterDescriptor = new ParameterDescriptor.Builder().name(parameterName).build();
-                    parameters.remove(parameterDescriptor);
                     changeAffectingComponents = true;
+
+                    final ParameterDescriptor parameterDescriptor = new ParameterDescriptor.Builder().name(parameterName).build();
+                    final Parameter oldParameter = parameters.remove(parameterDescriptor);
+
+                    parameterUpdates.put(parameterName, new StandardParameterUpdate(parameterName, oldParameter.getValue(), null, parameterDescriptor.isSensitive()));
                 } else {
                     final Parameter updatedParameter = createFullyPopulatedParameter(parameter);
 
                     final Parameter oldParameter = parameters.put(updatedParameter.getDescriptor(), updatedParameter);
                     if (oldParameter == null || !Objects.equals(oldParameter.getValue(), updatedParameter.getValue())) {
                         changeAffectingComponents = true;
+
+                        final String previousValue = oldParameter == null ? null : oldParameter.getValue();
+                        parameterUpdates.put(parameterName, new StandardParameterUpdate(parameterName, previousValue, updatedParameter.getValue(), updatedParameter.getDescriptor().isSensitive()));
                     }
                 }
             }
 
             if (changeAffectingComponents) {
+                logger.debug("Parameter Context {} was updated. {} parameters changed ({}). Notifying all affected components.", this, parameterUpdates.size(), parameterUpdates);
+
                 for (final ProcessGroup processGroup : parameterReferenceManager.getProcessGroupsBound(this)) {
                     try {
-                        processGroup.onParameterContextUpdated();
+                        processGroup.onParameterContextUpdated(parameterUpdates);
                     } catch (final Exception e) {
                         logger.error("Failed to notify {} that Parameter Context was updated", processGroup, e);
                     }
                 }
+            } else {
+                logger.debug("Parameter Context {} was updated. {} parameters changed ({}). No existing components are affected.", this, parameterUpdates.size(), parameterUpdates);
             }
         } finally {
             writeLock.unlock();
@@ -325,6 +337,11 @@ public class StandardParameterContext implements ParameterContext {
     }
 
     @Override
+    public String toString() {
+        return "StandardParameterContext[name=" + name + "]";
+    }
+
+    @Override
     public Authorizable getParentAuthorizable() {
         return new Authorizable() {
             @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterUpdate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterUpdate.java
new file mode 100644
index 0000000..afb963a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/parameter/StandardParameterUpdate.java
@@ -0,0 +1,83 @@
+/*
+ * 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.parameter;
+
+import java.util.Objects;
+
+public class StandardParameterUpdate implements ParameterUpdate {
+    private final String parameterName;
+    private final String previousValue;
+    private final String updatedValue;
+    private final boolean sensitiveParameter;
+
+    public StandardParameterUpdate(final String parameterName, final String previousValue, final String updatedValue, final boolean sensitiveParameter) {
+        this.parameterName = parameterName;
+        this.previousValue = previousValue;
+        this.updatedValue = updatedValue;
+        this.sensitiveParameter = sensitiveParameter;
+    }
+
+    @Override
+    public String getParameterName() {
+        return parameterName;
+    }
+
+    @Override
+    public String getPreviousValue() {
+        return previousValue;
+    }
+
+    @Override
+    public String getUpdatedValue() {
+        return updatedValue;
+    }
+
+    @Override
+    public boolean isSensitive() {
+        return sensitiveParameter;
+    }
+
+    @Override
+    public String toString() {
+        if (sensitiveParameter) {
+            return "StandardParameterUpdate[parameterName=" + parameterName + ", sensitive=true]";
+        } else {
+            return "StandardParameterUpdate[parameterName=" + parameterName + ", sensitive=false, previous value='" + previousValue + "', updated value='" + updatedValue + "']";
+        }
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o){
+            return true;
+        }
+
+        if (!(o instanceof StandardParameterUpdate)) {
+            return false;
+        }
+
+        final StandardParameterUpdate that = (StandardParameterUpdate) o;
+        return Objects.equals(parameterName, that.parameterName)
+            && Objects.equals(previousValue, that.previousValue)
+            && Objects.equals(updatedValue, that.updatedValue);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(parameterName, previousValue, updatedValue);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
index e2e4561..3017faa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
@@ -36,6 +36,7 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.ProcessGroupCounts;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterUpdate;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.flow.VersionControlInformation;
@@ -705,7 +706,7 @@ public class MockProcessGroup implements ProcessGroup {
     }
 
     @Override
-    public void onParameterContextUpdated() {
+    public void onParameterContextUpdated(final Map<String, ParameterUpdate> updatedParameters) {
     }
 
     @Override


[nifi] 42/47: NIFI-7256: This closes #4142. Fixed thresholds in unit test. Instead of assuming that multiple runs of the processor will occur within 100 milliseconds, allowed the multiple runs to occur within 3 mins of one another.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit c3457da96693f6323af013062135e270cf60ea24
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Fri Mar 13 13:11:27 2020 -0400

    NIFI-7256: This closes #4142. Fixed thresholds in unit test. Instead of assuming that multiple runs of the processor will occur within 100 milliseconds, allowed the multiple runs to occur within 3 mins of one another.
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../processors/standard/TestMonitorActivity.java   | 61 +++++++++++-----------
 1 file changed, 30 insertions(+), 31 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMonitorActivity.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMonitorActivity.java
index bd375e4..aa2d289 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMonitorActivity.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMonitorActivity.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
 import org.apache.nifi.components.state.Scope;
@@ -39,7 +40,7 @@ import static org.junit.Assert.assertTrue;
 public class TestMonitorActivity {
 
     @Test
-    public void testFirstMessage() throws InterruptedException, IOException {
+    public void testFirstMessage() throws InterruptedException {
         final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(1000L));
         runner.setProperty(MonitorActivity.CONTINUALLY_SEND_MESSAGES, "false");
         runner.setProperty(MonitorActivity.THRESHOLD, "100 millis");
@@ -255,7 +256,7 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorInvalidReportingNode() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(100));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
 
         runner.setClustered(true);
         runner.setPrimaryNode(false);
@@ -267,7 +268,7 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActive() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(100));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
@@ -289,7 +290,7 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActiveFallbackToNodeScope() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(100));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(false);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
@@ -309,7 +310,7 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActiveWithLatestTimestamp() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(100));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
@@ -343,7 +344,7 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActiveMoreRecentTimestampExisted() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(100));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
@@ -377,7 +378,7 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActiveCopyAttribute() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(100));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
@@ -402,11 +403,11 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorInactivity() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(10000));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -425,11 +426,11 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorInactivityFallbackToNodeScope() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(10000));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(false);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -448,14 +449,14 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorInactivityOnPrimaryNode() throws Exception {
-        final TestableProcessor processor = new TestableProcessor(10000);
+        final TestableProcessor processor = new TestableProcessor(TimeUnit.MINUTES.toMillis(120));
 
         final TestRunner runner = TestRunners.newTestRunner(processor);
         runner.setClustered(true);
         runner.setPrimaryNode(true);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
         runner.setProperty(MonitorActivity.REPORTING_NODE, MonitorActivity.REPORT_NODE_PRIMARY);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -474,12 +475,12 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorInactivityOnNode() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(10000));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
         runner.setProperty(MonitorActivity.REPORTING_NODE, MonitorActivity.REPORT_NODE_PRIMARY);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive, but this not shouldn't send flow file
@@ -493,11 +494,11 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActivityRestoredBySelf() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(10000));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -529,12 +530,12 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActivityRestoredBySelfOnNode() throws Exception {
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(10000));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
         runner.setProperty(MonitorActivity.REPORTING_NODE, MonitorActivity.REPORT_NODE_PRIMARY);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -563,14 +564,14 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActivityRestoredBySelfOnPrimaryNode() throws Exception {
-        final TestableProcessor processor = new TestableProcessor(10000);
+        final TestableProcessor processor = new TestableProcessor(TimeUnit.MINUTES.toMillis(120));
 
         final TestRunner runner = TestRunners.newTestRunner(processor);
         runner.setClustered(true);
         runner.setPrimaryNode(true);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
         runner.setProperty(MonitorActivity.REPORTING_NODE, MonitorActivity.REPORT_NODE_PRIMARY);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -602,14 +603,14 @@ public class TestMonitorActivity {
 
     @Test
     public void testClusterMonitorActivityRestoredBySelfOnPrimaryNodeFallbackToNodeScope() throws Exception {
-        final TestableProcessor processor = new TestableProcessor(10000);
+        final TestableProcessor processor = new TestableProcessor(TimeUnit.MINUTES.toMillis(120));
 
         final TestRunner runner = TestRunners.newTestRunner(processor);
         runner.setClustered(false);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
         runner.setProperty(MonitorActivity.REPORTING_NODE, MonitorActivity.REPORT_NODE_PRIMARY);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -640,11 +641,11 @@ public class TestMonitorActivity {
     @Test
     public void testClusterMonitorActivityRestoredByOtherNode() throws Exception {
 
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(10000));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -674,14 +675,14 @@ public class TestMonitorActivity {
     @Test
     public void testClusterMonitorActivityRestoredByOtherNodeOnPrimary() throws Exception {
 
-        final TestableProcessor processor = new TestableProcessor(10000);
+        final TestableProcessor processor = new TestableProcessor(TimeUnit.MINUTES.toMillis(120));
 
         final TestRunner runner = TestRunners.newTestRunner(processor);
         runner.setClustered(true);
         runner.setPrimaryNode(true);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
         runner.setProperty(MonitorActivity.REPORTING_NODE, MonitorActivity.REPORT_NODE_PRIMARY);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "1 hour");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive
@@ -705,18 +706,16 @@ public class TestMonitorActivity {
         assertEquals("value1", activityRestoredFiles.get(0).getAttribute("key1"));
         assertEquals("value2", activityRestoredFiles.get(0).getAttribute("key2"));
         runner.clearTransferState();
-
     }
 
     @Test
     public void testClusterMonitorActivityRestoredByOtherNodeOnNode() throws Exception {
-
-        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(10000));
+        final TestRunner runner = TestRunners.newTestRunner(new TestableProcessor(TimeUnit.MINUTES.toMillis(120)));
         runner.setClustered(true);
         runner.setPrimaryNode(false);
         runner.setProperty(MonitorActivity.MONITORING_SCOPE, MonitorActivity.SCOPE_CLUSTER);
         runner.setProperty(MonitorActivity.REPORTING_NODE, MonitorActivity.REPORT_NODE_PRIMARY);
-        runner.setProperty(MonitorActivity.THRESHOLD, "100 ms");
+        runner.setProperty(MonitorActivity.THRESHOLD, "3 mins");
         runner.setProperty(MonitorActivity.COPY_ATTRIBUTES, "true");
 
         // Becomes inactive


[nifi] 31/47: Fixed bug in JsonRowRecordReader when reading a 'raw' record with a schema that indicates that a field should be a Map. Also updated unit test to explicitly define schema, since schema inference will never return a Map but rather a Record

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 7839493dfaa3e97e75c376c113dd1176519308b2
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Fri Mar 6 11:45:59 2020 -0500

    Fixed bug in JsonRowRecordReader when reading a 'raw' record with a schema that indicates that a field should be a Map. Also updated unit test to explicitly define schema, since schema inference will never return a Map but rather a Record
---
 .../nifi/processors/standard/TestValidateRecord.java | 13 +++++++------
 .../nifi/json/AbstractJsonRowRecordReader.java       | 20 +++++++++++++++++++-
 2 files changed, 26 insertions(+), 7 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
index 1b4b7e4..74c3a17 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
@@ -33,7 +33,6 @@ import org.apache.nifi.schema.inference.SchemaInferenceUtil;
 import org.apache.nifi.serialization.DateTimeUtils;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.MockRecordWriter;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
@@ -53,7 +52,7 @@ import java.io.UnsupportedEncodingException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
-import java.util.HashMap;
+import java.util.Map;
 import java.util.Optional;
 
 import static org.junit.Assert.assertEquals;
@@ -560,6 +559,7 @@ public class TestValidateRecord {
         final JsonTreeReader jsonReader = new JsonTreeReader();
         runner.addControllerService("reader", jsonReader);
         runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, "schema-text-property");
+        runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY);
         runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_TEXT, validateSchema);
         runner.enableControllerService(jsonReader);
 
@@ -585,11 +585,12 @@ public class TestValidateRecord {
         try (final InputStream in = new ByteArrayInputStream(source); final AvroRecordReader reader = new AvroReaderWithEmbeddedSchema(in)) {
             final Object[] values = reader.nextRecord().getValues();
             assertEquals("uuid", values[0]);
-            assertEquals(2, ((HashMap<?,?>) values[1]).size());
+            assertEquals(2, ((Map<?,?>) values[1]).size());
             final Object[] data = (Object[]) values[2];
-            assertEquals(2, ( (HashMap<?,?>) ((MapRecord) data[0]).getValue("points")).size());
-            assertEquals(2, ( (HashMap<?,?>) ((MapRecord) data[1]).getValue("points")).size());
-            assertEquals(2, ( (HashMap<?,?>) ((MapRecord) data[2]).getValue("points")).size());
+            assertEquals(3, data.length);
+            assertEquals(2, ( (Map<?,?>) ((Record) data[0]).getValue("points")).size());
+            assertEquals(2, ( (Map<?,?>) ((Record) data[1]).getValue("points")).size());
+            assertEquals(2, ( (Map<?,?>) ((Record) data[2]).getValue("points")).size());
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
index f163707..6acb0d7 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
@@ -28,6 +28,7 @@ import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.type.ArrayDataType;
 import org.apache.nifi.serialization.record.type.ChoiceDataType;
+import org.apache.nifi.serialization.record.type.MapDataType;
 import org.apache.nifi.serialization.record.type.RecordDataType;
 import org.apache.nifi.serialization.record.util.DataTypeUtils;
 import org.codehaus.jackson.JsonFactory;
@@ -189,7 +190,24 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
 
         if (fieldNode.isObject()) {
             RecordSchema childSchema = null;
-            if (dataType != null && RecordFieldType.RECORD == dataType.getFieldType()) {
+            if (dataType != null && RecordFieldType.MAP == dataType.getFieldType()) {
+                final MapDataType mapDataType = (MapDataType) dataType;
+                final DataType valueType = mapDataType.getValueType();
+
+                final Map<String, Object> mapValue = new HashMap<>();
+
+                final Iterator<Map.Entry<String, JsonNode>> fieldItr = fieldNode.getFields();
+                while (fieldItr.hasNext()) {
+                    final Map.Entry<String, JsonNode> entry = fieldItr.next();
+                    final String elementName = entry.getKey();
+                    final JsonNode elementNode = entry.getValue();
+
+                    final Object nodeValue = getRawNodeValue(elementNode, valueType, fieldName + "['" + elementName + "']");
+                    mapValue.put(elementName, nodeValue);
+                }
+
+                return mapValue;
+            } else if (dataType != null && RecordFieldType.RECORD == dataType.getFieldType()) {
                 final RecordDataType recordDataType = (RecordDataType) dataType;
                 childSchema = recordDataType.getChildSchema();
             } else if (dataType != null && RecordFieldType.CHOICE == dataType.getFieldType()) {


[nifi] 19/47: NIFI-7231: move controller service validation out of synchronized block for enabling

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit ca27a6c5490e6f4473f8421cdfd16a024adc15fe
Author: Mark Bean <ma...@gmail.com>
AuthorDate: Fri Mar 6 17:18:09 2020 +0000

    NIFI-7231: move controller service validation out of synchronized block for enabling
    
    This closes #4118.
    
    Signed-off-by: Mark Payne <ma...@hotmail.com>
---
 .../nifi/components/validation/ValidationState.java      |  4 ++++
 .../nifi/controller/service/ServiceStateTransition.java  | 16 ----------------
 .../service/StandardControllerServiceNode.java           |  9 ++++++++-
 3 files changed, 12 insertions(+), 17 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java
index 17b3ff2..0f8f57f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java
@@ -37,4 +37,8 @@ public class ValidationState {
     public Collection<ValidationResult> getValidationErrors() {
         return validationErrors;
     }
+
+    public String toString() {
+        return status.toString();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
index 28f77b1..a2cd537 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
@@ -17,8 +17,6 @@
 
 package org.apache.nifi.controller.service;
 
-import org.apache.nifi.controller.ComponentNode;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
@@ -31,15 +29,10 @@ public class ServiceStateTransition {
     private final List<CompletableFuture<?>> enabledFutures = new ArrayList<>();
     private final List<CompletableFuture<?>> disabledFutures = new ArrayList<>();
 
-    private final ControllerServiceNode serviceNode;
     private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
     private final Lock writeLock = rwLock.writeLock();
     private final Lock readLock = rwLock.readLock();
 
-    public ServiceStateTransition(final ControllerServiceNode serviceNode) {
-        this.serviceNode = serviceNode;
-    }
-
     public boolean transitionToEnabling(final ControllerServiceState expectedState, final CompletableFuture<?> enabledFuture) {
         writeLock.lock();
         try {
@@ -64,8 +57,6 @@ public class ServiceStateTransition {
 
             state = ControllerServiceState.ENABLED;
 
-            validateReferences(serviceNode);
-
             enabledFutures.forEach(future -> future.complete(null));
             return true;
         } finally {
@@ -73,13 +64,6 @@ public class ServiceStateTransition {
         }
     }
 
-    private void validateReferences(final ControllerServiceNode service) {
-        final List<ComponentNode> referencingComponents = service.getReferences().findRecursiveReferences(ComponentNode.class);
-        for (final ComponentNode component : referencingComponents) {
-            component.performValidation();
-        }
-    }
-
     public boolean transitionToDisabling(final ControllerServiceState expectedState, final CompletableFuture<?> disabledFuture) {
         writeLock.lock();
         try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index da08c8c..e75fb3c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -111,7 +111,7 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
         this.serviceProvider = serviceProvider;
         this.active = new AtomicBoolean();
         setControllerServiceAndProxy(implementation, proxiedControllerService, invocationHandler);
-        stateTransition = new ServiceStateTransition(this);
+        stateTransition = new ServiceStateTransition();
     }
 
     @Override
@@ -436,6 +436,7 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
                         synchronized (active) {
                             shouldEnable = active.get() && stateTransition.enable(); // Transitioning the state to ENABLED will complete our future.
                         }
+                        validateReferences();
 
                         if (!shouldEnable) {
                             LOG.info("Disabling service {} after it has been enabled due to disable action being initiated.", service);
@@ -474,6 +475,12 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
         return future;
     }
 
+    private void validateReferences() {
+        final List<ComponentNode> referencingComponents = getReferences().findRecursiveReferences(ComponentNode.class);
+        for (final ComponentNode component : referencingComponents) {
+            component.performValidation();
+        }
+    }
 
     /**
      * Will atomically disable this service by invoking its @OnDisabled operation.


[nifi] 39/47: NIFI-7229 - Upgrade jackson-databind direct dependencies

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 1f2b68c92bd94bdb30c1095e9379cf167fcdc744
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Thu Mar 5 01:39:05 2020 +0100

    NIFI-7229 - Upgrade jackson-databind direct dependencies
    
    This closes #4113
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index 89cdb0a..01f0b12 100644
--- a/pom.xml
+++ b/pom.xml
@@ -92,7 +92,7 @@
         <ranger.version>2.0.0</ranger.version>
         <jetty.version>9.4.19.v20190610</jetty.version>
         <jackson.version>2.9.10</jackson.version>
-        <jackson-databind.version>2.9.10.1</jackson-databind.version>
+        <jackson-databind.version>2.9.10.3</jackson-databind.version>
         <nifi.registry.version>0.5.0</nifi.registry.version>
         <nifi.groovy.version>2.5.4</nifi.groovy.version>
         <surefire.version>2.22.0</surefire.version>


[nifi] 11/47: NIFI-4970 - EOF Exception in InvokeHttp when body's response is empty with gzip

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 1f41c21ab7b7b28d7416cfff6fc1db443c5961a7
Author: Eduardo Fontes <ed...@gmail.com>
AuthorDate: Tue Mar 3 23:38:12 2020 -0300

    NIFI-4970 - EOF Exception in InvokeHttp when body's response is empty with gzip
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4109.
---
 .../nifi/processors/standard/InvokeHTTP.java       |  2 +-
 .../nifi/processors/standard/TestInvokeHTTP.java   | 38 ++++++++++++++++++++++
 2 files changed, 39 insertions(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
index 78cadab..b5c853c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
@@ -842,7 +842,7 @@ public final class InvokeHTTP extends AbstractProcessor {
                 boolean outputBodyToRequestAttribute = (!isSuccess(statusCode) || putToAttribute) && requestFlowFile != null;
                 boolean outputBodyToResponseContent = (isSuccess(statusCode) && !putToAttribute) || context.getProperty(PROP_OUTPUT_RESPONSE_REGARDLESS).asBoolean();
                 ResponseBody responseBody = responseHttp.body();
-                boolean bodyExists = responseBody != null;
+                boolean bodyExists = responseBody != null ? responseBody.contentLength() > 0 : false;
 
                 InputStream responseBodyStream = null;
                 SoftLimitBoundedByteArrayOutputStream outputStreamToRequestAttribute = null;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
index ca04802..1a9ee10 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
@@ -268,4 +268,42 @@ public class TestInvokeHTTP extends TestInvokeHttpCommon {
         assertNull(regexAttributesToSendField.get(processor));
 
     }
+    @Test
+    public void testEmptyGzipHttpReponse() throws Exception {
+        addHandler(new EmptyGzipResponseHandler());
+
+        runner.setProperty(InvokeHTTP.PROP_URL, url);
+
+        createFlowFiles(runner);
+
+        runner.run();
+
+        runner.assertTransferCount(InvokeHTTP.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.REL_RESPONSE, 1);
+        runner.assertTransferCount(InvokeHTTP.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
+        runner.assertPenalizeCount(0);
+
+        //expected empty content in response FlowFile
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.REL_RESPONSE).get(0);
+        bundle.assertContentEquals(new byte[0]);
+        bundle.assertAttributeEquals(InvokeHTTP.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+        bundle.assertAttributeEquals("Content-Type", "text/plain");
+    }
+
+    public static class EmptyGzipResponseHandler extends AbstractHandler {
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+            response.setStatus(200);
+            response.setContentLength(0);
+            response.setContentType("text/plain");
+            response.setHeader("Content-Encoding", "gzip");
+        }
+
+    }
 }


[nifi] 16/47: NIFI-7232 if users do not supply a remote path PutSFTP with conflict resolution will fail with an NPE

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 5c3f9836fdde4bc47962454b06ba37d179c00960
Author: Joe Witt <jo...@apache.org>
AuthorDate: Fri Mar 6 10:32:12 2020 -0500

    NIFI-7232 if users do not supply a remote path PutSFTP with conflict resolution will fail with an NPE
---
 .../java/org/apache/nifi/processors/standard/PutFileTransfer.java    | 5 +++--
 .../java/org/apache/nifi/processors/standard/util/SFTPTransfer.java  | 1 +
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
index 20e2c5a..3b9af1a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
@@ -39,6 +39,7 @@ import org.apache.nifi.processors.standard.util.FileInfo;
 import org.apache.nifi.processors.standard.util.FileTransfer;
 import org.apache.nifi.processors.standard.util.SFTPTransfer;
 import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
 
 /**
  * Base class for PutFTP & PutSFTP
@@ -102,8 +103,8 @@ public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractPr
             do {
                 final String rootPath = context.getProperty(FileTransfer.REMOTE_PATH).evaluateAttributeExpressions(flowFile).getValue();
                 final String workingDirPath;
-                if (rootPath == null) {
-                    workingDirPath = null;
+                if (StringUtils.isBlank(rootPath)) {
+                    workingDirPath = transfer.getHomeDirectory(flowFile);
                 } else {
                     workingDirPath = transfer.getAbsolutePath(flowFile, rootPath);
                 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
index 3a341bc..870867f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
@@ -631,6 +631,7 @@ public class SFTPTransfer implements FileTransfer {
         try {
             this.homeDir = sftpClient.canonicalize("");
         } catch (IOException e) {
+            this.homeDir = "";
             // For some combination of server configuration and user home directory, getHome() can fail with "2: File not found"
             // Since  homeDir is only used tor SEND provenance event transit uri, this is harmless. Log and continue.
             logger.debug("Failed to retrieve {} home directory due to {}", new Object[]{username, e.getMessage()});


[nifi] 17/47: Fixed unit test failed with NIFI-7232

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 38124f2337eeeefd89fe957dad7c383a595f4057
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Fri Mar 6 11:46:16 2020 -0500

    Fixed unit test failed with NIFI-7232
    
    This closes #4117.
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../nifi/provenance/index/lucene/TestLuceneEventIndex.java   | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
index 69be483..d3733bd 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
@@ -40,6 +40,7 @@ import org.apache.nifi.provenance.store.ArrayListEventStore;
 import org.apache.nifi.provenance.store.EventStore;
 import org.apache.nifi.provenance.store.StorageResult;
 import org.apache.nifi.util.Tuple;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -83,6 +84,11 @@ public class TestLuceneEventIndex {
         return System.getProperty("os.name").toLowerCase().startsWith("windows");
     }
 
+    @Before
+    public void setup() {
+        idGenerator.set(0L);
+    }
+
     @Test
     public void testGetTimeRange() {
         final long now = System.currentTimeMillis();
@@ -170,7 +176,7 @@ public class TestLuceneEventIndex {
     }
 
     @Test(timeout = 60000)
-    public void testUnauthorizedEventsGetPlaceholdersForExpandChildren() throws InterruptedException {
+    public void testUnauthorizedEventsGetPlaceholdersForExpandChildren() throws InterruptedException, IOException {
         assumeFalse(isWindowsEnvironment());
         final RepositoryConfiguration repoConfig = createConfig(1);
         repoConfig.setDesiredIndexSize(1L);
@@ -224,13 +230,15 @@ public class TestLuceneEventIndex {
 
         List<LineageNode> nodes = Collections.emptyList();
         while (nodes.size() < 5) {
-            final ComputeLineageSubmission submission = index.submitExpandChildren(1L, user, allowForkEvents);
+            final ComputeLineageSubmission submission = index.submitExpandChildren(fork.getEventId(), user, allowForkEvents);
             assertTrue(submission.getResult().awaitCompletion(15, TimeUnit.SECONDS));
 
             nodes = submission.getResult().getNodes();
             Thread.sleep(25L);
         }
 
+        nodes.forEach(System.out::println);
+
         assertEquals(5, nodes.size());
 
         assertEquals(1L, nodes.stream().filter(n -> n.getNodeType() == LineageNodeType.FLOWFILE_NODE).count());


[nifi] 08/47: NIFI-5644 Fixed typo in getWrappedQuery method of AbstractDatabaseFetchProcessor class

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 91f3eb04677a409101577626eba2ad1ea7df17f9
Author: karthik.kadajji <ka...@ultratendency.com>
AuthorDate: Tue Mar 3 13:17:04 2020 +0100

    NIFI-5644 Fixed typo in getWrappedQuery method of AbstractDatabaseFetchProcessor class
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4106.
---
 .../nifi/processors/standard/AbstractDatabaseFetchProcessor.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
index a99ca6a..dbf8ea3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
@@ -316,8 +316,8 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
         }
     }
 
-    protected static StringBuilder getWrappedQuery(DatabaseAdapter dbAdaper, String sqlQuery, String tableName) {
-       return new StringBuilder("SELECT * FROM (" + sqlQuery + ") " + dbAdaper.getTableAliasClause(tableName));
+    protected static StringBuilder getWrappedQuery(DatabaseAdapter dbAdapter, String sqlQuery, String tableName) {
+       return new StringBuilder("SELECT * FROM (" + sqlQuery + ") " + dbAdapter.getTableAliasClause(tableName));
     }
 
     protected static String getMaxValueFromRow(ResultSet resultSet,


[nifi] 03/47: NIFI-7183 - This closes #4073. Improve ReplaceText when removing FF's content

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit fa0632b26dcd67d36ae7ae35023ce2b3a7af375c
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Fri Feb 21 18:35:27 2020 -0800

    NIFI-7183 - This closes #4073. Improve ReplaceText when removing FF's content
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../nifi/processors/standard/ReplaceText.java      | 37 ++++++++++++++--------
 1 file changed, 24 insertions(+), 13 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
index 773458b..a75828a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
@@ -26,6 +26,7 @@ import org.apache.nifi.annotation.behavior.SystemResource;
 import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 import org.apache.nifi.attribute.expression.language.exception.IllegalAttributeException;
 import org.apache.nifi.components.AllowableValue;
@@ -199,6 +200,7 @@ public class ReplaceText extends AbstractProcessor {
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
+    private ReplacementStrategyExecutor replacementStrategyExecutor;
 
     @Override
     protected void init(final ProcessorInitializationContext context) {
@@ -254,21 +256,12 @@ public class ReplaceText extends AbstractProcessor {
         return errors;
     }
 
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
-        FlowFile flowFile = session.get();
-        if (flowFile == null) {
-            return;
-        }
-
-        final ComponentLog logger = getLogger();
-
+    @OnScheduled
+    public void setup(ProcessContext context) {
         final String replacementStrategy = context.getProperty(REPLACEMENT_STRATEGY).getValue();
-
-        final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
+        final String evaluateMode = context.getProperty(EVALUATION_MODE).getValue();
         final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
 
-        final String evaluateMode = context.getProperty(EVALUATION_MODE).getValue();
         final byte[] buffer;
         if (replacementStrategy.equalsIgnoreCase(regexReplaceValue) || replacementStrategy.equalsIgnoreCase(literalReplaceValue)) {
             buffer = new byte[maxBufferSize];
@@ -276,7 +269,6 @@ public class ReplaceText extends AbstractProcessor {
             buffer = null;
         }
 
-        ReplacementStrategyExecutor replacementStrategyExecutor;
         switch (replacementStrategy) {
             case prependValue:
                 replacementStrategyExecutor = new PrependReplace();
@@ -288,6 +280,10 @@ public class ReplaceText extends AbstractProcessor {
                 // for backward compatibility - if replacement regex is ".*" then we will simply always replace the content.
                 if (context.getProperty(SEARCH_VALUE).getValue().equals(".*")) {
                     replacementStrategyExecutor = new AlwaysReplace();
+                } else if (context.getProperty(SEARCH_VALUE).getValue().equals(DEFAULT_REGEX)
+                        && evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)
+                        && context.getProperty(REPLACEMENT_VALUE).getValue().isEmpty()) {
+                    replacementStrategyExecutor = new AlwaysReplace();
                 } else {
                     replacementStrategyExecutor = new RegexReplace(buffer, context);
                 }
@@ -302,6 +298,20 @@ public class ReplaceText extends AbstractProcessor {
             default:
                 throw new AssertionError();
         }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
+        final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
+        final String evaluateMode = context.getProperty(EVALUATION_MODE).getValue();
 
         if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
             if (flowFile.getSize() > maxBufferSize && replacementStrategyExecutor.isAllDataBufferedForEntireText()) {
@@ -325,6 +335,7 @@ public class ReplaceText extends AbstractProcessor {
             session.transfer(flowFile, REL_FAILURE);
             return;
         }
+
         logger.info("Transferred {} to 'success'", new Object[] {flowFile});
         session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
         session.transfer(flowFile, REL_SUCCESS);


[nifi] 15/47: NIFI-7226: Add Connection Factory configuration properties to PublishJMS and ConsumeJMS processors

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit ad892a4e27ebf937a0350b897a6179c85c5ac809
Author: Peter Turcsanyi <tu...@apache.org>
AuthorDate: Wed Mar 4 19:11:43 2020 +0100

    NIFI-7226: Add Connection Factory configuration properties to PublishJMS and ConsumeJMS processors
    
    Some JMS client libraries may not work with the existing controller services due to incompatible
    classloader handling between the 3rd party library and NiFi.
    Via configuring the Connection Factory on the processor itself, only the processor's and its
    children's classloaders will be used which eliminates the mentioned incompatibility.
    
    This closes #4110.
    
    Signed-off-by: Mark Payne <ma...@hotmail.com>
---
 ...ion.java => IJMSConnectionFactoryProvider.java} |   7 +-
 .../cf/JMSConnectionFactoryProviderDefinition.java |  25 +-
 .../nifi/jms/cf/JMSConnectionFactoryHandler.java   | 248 +++++++++++++++++
 .../jms/cf/JMSConnectionFactoryProperties.java     | 113 ++++++++
 .../nifi/jms/cf/JMSConnectionFactoryProvider.java  | 307 ++-------------------
 .../jms/cf/JndiJmsConnectionFactoryHandler.java    | 154 +++++++++++
 .../jms/cf/JndiJmsConnectionFactoryProperties.java | 114 ++++++++
 .../jms/cf/JndiJmsConnectionFactoryProvider.java   | 191 ++-----------
 .../nifi/jms/processors/AbstractJMSProcessor.java  | 213 ++++++++++----
 .../org/apache/nifi/jms/processors/ConsumeJMS.java |  33 ++-
 .../org/apache/nifi/jms/processors/PublishJMS.java |  65 +++++
 .../additionalDetails.html                         |  19 +-
 .../additionalDetails.html                         |  23 +-
 .../additionalDetails.html                         |  25 +-
 .../additionalDetails.html                         |  25 +-
 ...ava => JMSConnectionFactoryHandlerForTest.java} |  23 +-
 .../cf/JMSConnectionFactoryProviderForTest.java    |  20 +-
 .../jms/cf/JMSConnectionFactoryProviderTest.java   | 201 +++++++-------
 .../jms/processors/ConnectionFactoryConfigIT.java  | 176 ++++++++++++
 .../ConnectionFactoryConfigValidatorTest.java      | 145 ++++++++++
 20 files changed, 1423 insertions(+), 704 deletions(-)

diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/IJMSConnectionFactoryProvider.java
similarity index 87%
copy from nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java
copy to nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/IJMSConnectionFactoryProvider.java
index 6bab920..338224a 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/IJMSConnectionFactoryProvider.java
@@ -18,18 +18,15 @@ package org.apache.nifi.jms.cf;
 
 import javax.jms.ConnectionFactory;
 
-import org.apache.nifi.controller.ControllerService;
-
 /**
  * Defines a strategy to create implementations to load and initialize third
  * party implementations of the {@link ConnectionFactory}
  */
-public interface JMSConnectionFactoryProviderDefinition extends ControllerService {
+public interface IJMSConnectionFactoryProvider {
 
     /**
      * Returns an instance of the {@link ConnectionFactory} specific to the
-     * target messaging system (i.e.,
-     * org.apache.activemq.ActiveMQConnectionFactory).
+     * target messaging system (eg. org.apache.activemq.ActiveMQConnectionFactory).
      *
      * @return instance of {@link ConnectionFactory}
      */
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java
index 6bab920..be4d1b7 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java
@@ -16,31 +16,10 @@
  */
 package org.apache.nifi.jms.cf;
 
-import javax.jms.ConnectionFactory;
-
 import org.apache.nifi.controller.ControllerService;
 
 /**
- * Defines a strategy to create implementations to load and initialize third
- * party implementations of the {@link ConnectionFactory}
+ * Base interface of controller service implementations of IJMSConnectionFactoryProvider.
  */
-public interface JMSConnectionFactoryProviderDefinition extends ControllerService {
-
-    /**
-     * Returns an instance of the {@link ConnectionFactory} specific to the
-     * target messaging system (i.e.,
-     * org.apache.activemq.ActiveMQConnectionFactory).
-     *
-     * @return instance of {@link ConnectionFactory}
-     */
-    ConnectionFactory getConnectionFactory();
-
-    /**
-     * Resets {@link ConnectionFactory}.
-     * Provider should reset {@link ConnectionFactory} only if a copy provided by a client matches
-     * current {@link ConnectionFactory}.
-     * @param cachedFactory - {@link ConnectionFactory} cached by client.
-     */
-    void resetConnectionFactory(ConnectionFactory cachedFactory);
-
+public interface JMSConnectionFactoryProviderDefinition extends IJMSConnectionFactoryProvider, ControllerService {
 }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java
new file mode 100644
index 0000000..4702e89
--- /dev/null
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java
@@ -0,0 +1,248 @@
+/*
+ * 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.jms.cf;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+
+import javax.jms.ConnectionFactory;
+import javax.net.ssl.SSLContext;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_BROKER_URI;
+import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL;
+import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_SSL_CONTEXT_SERVICE;
+
+/**
+ * Handler class to create a JMS Connection Factory by instantiating the vendor specific javax.jms.ConnectionFactory
+ * implementation class and configuring the Connection Factory object directly.
+ * The handler can be used from controller services and processors as well.
+ */
+public class JMSConnectionFactoryHandler implements IJMSConnectionFactoryProvider {
+
+    private final PropertyContext context;
+    private final Set<PropertyDescriptor> propertyDescriptors;
+    private final ComponentLog logger;
+
+    public JMSConnectionFactoryHandler(ConfigurationContext context, ComponentLog logger) {
+        this.context = context;
+        this.propertyDescriptors = context.getProperties().keySet();
+        this.logger = logger;
+    }
+
+    public JMSConnectionFactoryHandler(ProcessContext context, ComponentLog logger) {
+        this.context = context;
+        this.propertyDescriptors = context.getProperties().keySet();
+        this.logger = logger;
+    }
+
+    private volatile ConnectionFactory connectionFactory;
+
+    @Override
+    public synchronized ConnectionFactory getConnectionFactory() {
+        if (connectionFactory == null) {
+            initConnectionFactory();
+        } else {
+            logger.debug("Connection Factory has already been initialized. Will return cached instance.");
+        }
+
+        return connectionFactory;
+    }
+
+    @Override
+    public synchronized void resetConnectionFactory(ConnectionFactory cachedFactory) {
+        if (cachedFactory == connectionFactory) {
+            logger.debug("Resetting connection factory");
+            connectionFactory = null;
+        }
+    }
+
+    private void initConnectionFactory() {
+        try {
+            if (logger.isInfoEnabled()) {
+                logger.info("Configuring " + getClass().getSimpleName() + " for '"
+                        + context.getProperty(JMS_CONNECTION_FACTORY_IMPL).evaluateAttributeExpressions().getValue() + "' to be connected to '"
+                        + context.getProperty(JMS_BROKER_URI).evaluateAttributeExpressions().getValue() + "'");
+            }
+
+            createConnectionFactoryInstance();
+            setConnectionFactoryProperties();
+        } catch (Exception e) {
+            connectionFactory = null;
+            logger.error("Failed to configure " + getClass().getSimpleName(), e);
+            throw new IllegalStateException(e);
+        }
+    }
+
+    /**
+     * Creates an instance of the {@link ConnectionFactory} from the provided
+     * 'CONNECTION_FACTORY_IMPL'.
+     */
+    private void createConnectionFactoryInstance() {
+        String connectionFactoryImplName = context.getProperty(JMS_CONNECTION_FACTORY_IMPL).evaluateAttributeExpressions().getValue();
+        connectionFactory = Utils.newDefaultInstance(connectionFactoryImplName);
+    }
+
+    /**
+     * This operation follows standard bean convention by matching property name
+     * to its corresponding 'setter' method. Once the method was located it is
+     * invoked to set the corresponding property to a value provided by during
+     * service configuration. For example, 'channel' property will correspond to
+     * 'setChannel(..) method and 'queueManager' property will correspond to
+     * setQueueManager(..) method with a single argument. The bean convention is also
+     * explained in user manual for this component with links pointing to
+     * documentation of various ConnectionFactories.
+     * <p>
+     * There are also few adjustments to accommodate well known brokers. For
+     * example ActiveMQ ConnectionFactory accepts address of the Message Broker
+     * in a form of URL while IBMs in the form of host/port pair(s).
+     * <p>
+     * This method will use the value retrieved from the 'BROKER_URI' static
+     * property as is. An exception to this if ConnectionFactory implementation
+     * is coming from IBM MQ and connecting to a stand-alone queue manager. In
+     * this case the Broker URI is expected to be entered as a colon separated
+     * host/port pair, which then is split on ':' and the resulting pair will be
+     * used to execute setHostName(..) and setPort(..) methods on the provided
+     * ConnectionFactory.
+     * <p>
+     * This method may need to be maintained and adjusted to accommodate other
+     * implementation of ConnectionFactory, but only for URL/Host/Port issue.
+     * All other properties are set as dynamic properties where user essentially
+     * provides both property name and value.
+     *
+     * @see <a href="http://activemq.apache.org/maven/apidocs/org/apache/activemq/ActiveMQConnectionFactory.html#setBrokerURL-java.lang.String-">setBrokerURL(String brokerURL)</a>
+     * @see <a href="https://docs.tibco.com/pub/enterprise_message_service/8.1.0/doc/html/tib_ems_api_reference/api/javadoc/com/tibco/tibjms/TibjmsConnectionFactory.html#setServerUrl(java.lang.String)">setServerUrl(String serverUrl)</a>
+     * @see <a href="https://www.ibm.com/support/knowledgecenter/en/SSFKSJ_7.1.0/com.ibm.mq.javadoc.doc/WMQJMSClasses/com/ibm/mq/jms/MQConnectionFactory.html#setHostName_java.lang.String_">setHostName(String hostname)</a>
+     * @see <a href="https://www.ibm.com/support/knowledgecenter/en/SSFKSJ_7.1.0/com.ibm.mq.javadoc.doc/WMQJMSClasses/com/ibm/mq/jms/MQConnectionFactory.html#setPort_int_">setPort(int port)</a>
+     * @see <a href="https://www.ibm.com/support/knowledgecenter/en/SSFKSJ_7.1.0/com.ibm.mq.javadoc.doc/WMQJMSClasses/com/ibm/mq/jms/MQConnectionFactory.html#setConnectionNameList_java.lang.String_">setConnectionNameList(String hosts)</a>
+     * @see #setProperty(String propertyName, Object propertyValue)
+     */
+    void setConnectionFactoryProperties() {
+        if (context.getProperty(JMS_BROKER_URI).isSet()) {
+            String brokerValue = context.getProperty(JMS_BROKER_URI).evaluateAttributeExpressions().getValue();
+            String connectionFactoryValue = context.getProperty(JMS_CONNECTION_FACTORY_IMPL).evaluateAttributeExpressions().getValue();
+            if (connectionFactoryValue.startsWith("org.apache.activemq")) {
+                setProperty("brokerURL", brokerValue);
+            } else if (connectionFactoryValue.startsWith("com.tibco.tibjms")) {
+                setProperty("serverUrl", brokerValue);
+            } else {
+                String[] brokerList = brokerValue.split(",");
+                if (connectionFactoryValue.startsWith("com.ibm.mq.jms")) {
+                    List<String> ibmConList = new ArrayList<String>();
+                    for (String broker : brokerList) {
+                        String[] hostPort = broker.split(":");
+                        if (hostPort.length == 2) {
+                            ibmConList.add(hostPort[0] + "(" + hostPort[1] + ")");
+                        } else {
+                            ibmConList.add(broker);
+                        }
+                    }
+                    setProperty("connectionNameList", String.join(",", ibmConList));
+                } else {
+                    // Try to parse broker URI as colon separated host/port pair. Use first pair if multiple given.
+                    String[] hostPort = brokerList[0].split(":");
+                    if (hostPort.length == 2) {
+                        // If broker URI indeed was colon separated host/port pair
+                        setProperty("hostName", hostPort[0]);
+                        setProperty("port", hostPort[1]);
+                    }
+                }
+            }
+        }
+
+        SSLContextService sc = context.getProperty(JMS_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+        if (sc != null) {
+            SSLContext ssl = sc.createSSLContext(ClientAuth.NONE);
+            setProperty("sSLSocketFactory", ssl.getSocketFactory());
+        }
+
+        propertyDescriptors.stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .forEach(descriptor -> {
+                    String propertyName = descriptor.getName();
+                    String propertyValue = context.getProperty(descriptor).evaluateAttributeExpressions().getValue();
+                    setProperty(propertyName, propertyValue);
+                });
+    }
+
+    /**
+     * Sets corresponding {@link ConnectionFactory}'s property to a
+     * 'propertyValue' by invoking a 'setter' method that corresponds to
+     * 'propertyName'. For example, 'channel' property will correspond to
+     * 'setChannel(..) method and 'queueManager' property will correspond to
+     * setQueueManager(..) method with a single argument.
+     * <p>
+     * NOTE: There is a limited type conversion to accommodate property value
+     * types since all NiFi configuration properties comes as String. It is
+     * accomplished by checking the argument type of the method and executing
+     * its corresponding conversion to target primitive (e.g., value 'true' will
+     * go thru Boolean.parseBoolean(propertyValue) if method argument is of type
+     * boolean). None-primitive values are not supported at the moment and will
+     * result in {@link IllegalArgumentException}. It is OK though since based
+     * on analysis of several ConnectionFactory implementation the all seem to
+     * follow bean convention and all their properties using Java primitives as
+     * arguments.
+     */
+    void setProperty(String propertyName, Object propertyValue) {
+        String methodName = toMethodName(propertyName);
+        Method[] methods = Utils.findMethods(methodName, connectionFactory.getClass());
+        if (methods != null && methods.length > 0) {
+            try {
+                for (Method method : methods) {
+                    Class<?> returnType = method.getParameterTypes()[0];
+                    if (String.class.isAssignableFrom(returnType)) {
+                        method.invoke(connectionFactory, propertyValue);
+                        return;
+                    } else if (int.class.isAssignableFrom(returnType)) {
+                        method.invoke(connectionFactory, Integer.parseInt((String) propertyValue));
+                        return;
+                    } else if (long.class.isAssignableFrom(returnType)) {
+                        method.invoke(connectionFactory, Long.parseLong((String) propertyValue));
+                        return;
+                    } else if (boolean.class.isAssignableFrom(returnType)) {
+                        method.invoke(connectionFactory, Boolean.parseBoolean((String) propertyValue));
+                        return;
+                    }
+                }
+                methods[0].invoke(connectionFactory, propertyValue);
+            } catch (Exception e) {
+                throw new IllegalStateException("Failed to set property " + propertyName, e);
+            }
+        } else if (propertyName.equals("hostName")) {
+            setProperty("host", propertyValue); // try 'host' as another common convention.
+        }
+    }
+
+    /**
+     * Will convert propertyName to a method name following bean convention. For
+     * example, 'channel' property will correspond to 'setChannel method and
+     * 'queueManager' property will correspond to setQueueManager method name
+     */
+    private String toMethodName(String propertyName) {
+        char c[] = propertyName.toCharArray();
+        c[0] = Character.toUpperCase(c[0]);
+        return "set" + new String(c);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProperties.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProperties.java
new file mode 100644
index 0000000..f08535f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProperties.java
@@ -0,0 +1,113 @@
+/*
+ * 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.jms.cf;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class JMSConnectionFactoryProperties {
+
+    private static final String BROKER = "broker";
+    private static final String CF_IMPL = "cf";
+    private static final String CF_LIB = "cflib";
+
+    public static final PropertyDescriptor JMS_CONNECTION_FACTORY_IMPL = new PropertyDescriptor.Builder()
+            .name(CF_IMPL)
+            .displayName("JMS Connection Factory Implementation Class")
+            .description("The fully qualified name of the JMS ConnectionFactory implementation "
+                    + "class (eg. org.apache.activemq.ActiveMQConnectionFactory).")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor JMS_CLIENT_LIBRARIES = new PropertyDescriptor.Builder()
+            .name(CF_LIB)
+            .displayName("JMS Client Libraries")
+            .description("Path to the directory with additional resources (eg. JARs, configuration files etc.) to be added "
+                    + "to the classpath (defined as a comma separated list of values). Such resources typically represent target JMS client libraries "
+                    + "for the ConnectionFactory implementation.")
+            .required(false)
+            .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .dynamicallyModifiesClasspath(true)
+            .build();
+
+    public static final PropertyDescriptor JMS_BROKER_URI = new PropertyDescriptor.Builder()
+            .name(BROKER)
+            .displayName("JMS Broker URI")
+            .description("URI pointing to the network location of the JMS Message broker. Example for ActiveMQ: "
+                    + "'tcp://myhost:61616'. Examples for IBM MQ: 'myhost(1414)' and 'myhost01(1414),myhost02(1414)'.")
+            .required(false)
+            .addValidator(new NonEmptyBrokerURIValidator())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor JMS_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("SSL Context Service")
+            .displayName("JMS SSL Context Service")
+            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Arrays.asList(
+            JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL,
+            JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES,
+            JMSConnectionFactoryProperties.JMS_BROKER_URI,
+            JMSConnectionFactoryProperties.JMS_SSL_CONTEXT_SERVICE
+    );
+
+    public static List<PropertyDescriptor> getPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    public static PropertyDescriptor getDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .description("Specifies the value for '" + propertyDescriptorName
+                        + "' property to be set on the provided Connection Factory implementation.")
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+                .build();
+    }
+
+    /**
+     * {@link Validator} that ensures that brokerURI's length > 0 after EL
+     * evaluation
+     */
+    private static class NonEmptyBrokerURIValidator implements Validator {
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input)) {
+                return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build();
+            }
+            return StandardValidators.NON_EMPTY_VALIDATOR.validate(subject, input, context);
+        }
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java
index b8382a1..5da735f 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java
@@ -16,17 +16,6 @@
  */
 package org.apache.nifi.jms.cf;
 
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.stream.Collectors;
-
-import javax.jms.ConnectionFactory;
-import javax.net.ssl.SSLContext;
-
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
@@ -34,17 +23,12 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnDisabled;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.ssl.SSLContextService.ClientAuth;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
+import javax.jms.ConnectionFactory;
+import java.util.List;
 
 /**
  * Provides a factory service that creates and initializes
@@ -52,14 +36,14 @@ import org.slf4j.LoggerFactory;
  * <p>
  * It accomplishes it by adjusting current classpath by adding to it the
  * additional resources (i.e., JMS client libraries) provided by the user via
- * {@link #CLIENT_LIB_DIR_PATH}, allowing it then to create an instance of the
+ * {@link JMSConnectionFactoryProperties#JMS_CLIENT_LIBRARIES}, allowing it then to create an instance of the
  * target {@link ConnectionFactory} based on the provided
- * {@link #CONNECTION_FACTORY_IMPL} which can be than access via
+ * {@link JMSConnectionFactoryProperties#JMS_CONNECTION_FACTORY_IMPL} which can be than access via
  * {@link #getConnectionFactory()} method.
  */
 @Tags({"jms", "messaging", "integration", "queue", "topic", "publish", "subscribe"})
 @CapabilityDescription("Provides a generic service to create vendor specific javax.jms.ConnectionFactory implementations. "
-        + "ConnectionFactory can be served once this service is configured successfully")
+        + "The Connection Factory can be served once this service is configured successfully.")
 @DynamicProperty(name = "The name of a Connection Factory configuration property.", value = "The value of a given Connection Factory configuration property.",
         description = "The properties that are set following Java Beans convention where a property name is derived from the 'set*' method of the vendor "
                 + "specific ConnectionFactory's implementation. For example, 'com.ibm.mq.jms.MQConnectionFactory.setChannel(String)' would imply 'channel' "
@@ -68,289 +52,36 @@ import org.slf4j.LoggerFactory;
 @SeeAlso(classNames = {"org.apache.nifi.jms.processors.ConsumeJMS", "org.apache.nifi.jms.processors.PublishJMS"})
 public class JMSConnectionFactoryProvider extends AbstractControllerService implements JMSConnectionFactoryProviderDefinition {
 
-    private final Logger logger = LoggerFactory.getLogger(JMSConnectionFactoryProvider.class);
-
-    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS;
-
-    private volatile boolean configured;
-
-    private volatile ConnectionFactory connectionFactory;
-
-    private static final String BROKER = "broker";
-    private static final String CF_IMPL = "cf";
-    private static final String CF_LIB = "cflib";
-
-    public static final PropertyDescriptor CONNECTION_FACTORY_IMPL = new PropertyDescriptor.Builder()
-            .name(CF_IMPL)
-            .displayName("MQ ConnectionFactory Implementation")
-            .description("A fully qualified name of the JMS ConnectionFactory implementation "
-                    + "class (i.e., org.apache.activemq.ActiveMQConnectionFactory)")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .required(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-    public static final PropertyDescriptor CLIENT_LIB_DIR_PATH = new PropertyDescriptor.Builder()
-            .name(CF_LIB)
-            .displayName("MQ Client Libraries path (i.e. /usr/jms/lib)")
-            .description("Path to the directory with additional resources (i.e., JARs, configuration files etc.) to be added "
-                    + "to the classpath. Such resources typically represent target MQ client libraries for the "
-                    + "ConnectionFactory implementation. Required if target is not ActiveMQ.")
-            .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
-            .required(false)
-            .dynamicallyModifiesClasspath(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-
-    // ConnectionFactory specific properties
-    public static final PropertyDescriptor BROKER_URI = new PropertyDescriptor.Builder()
-            .name(BROKER)
-            .displayName("Broker URI")
-            .description("URI pointing to the network location of the JMS Message broker. Example for ActiveMQ: "
-                    + "'tcp://myhost:61616'. Examples for IBM MQ: 'myhost(1414)' and 'myhost01(1414),myhost02(1414)'")
-            .addValidator(new NonEmptyBrokerURIValidator())
-            .required(false)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
-
-    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
-            .name("SSL Context Service")
-            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
-            .required(false)
-            .identifiesControllerService(SSLContextService.class)
-            .build();
-
-    static {
-        PROPERTY_DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(CONNECTION_FACTORY_IMPL, CLIENT_LIB_DIR_PATH, BROKER_URI, SSL_CONTEXT_SERVICE));
-    }
+    protected volatile JMSConnectionFactoryHandler delegate;
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return PROPERTY_DESCRIPTORS;
+        return JMSConnectionFactoryProperties.getPropertyDescriptors();
     }
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder()
-                .description("Specifies the value for '" + propertyDescriptorName
-                        + "' property to be set on the provided ConnectionFactory implementation.")
-                .name(propertyDescriptorName)
-                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-                .dynamic(true)
-                .build();
-    }
-
-    @Override
-    public void resetConnectionFactory(ConnectionFactory cachedFactory) {
-        if (cachedFactory == connectionFactory) {
-            getLogger().debug("Resetting connection factory");
-            connectionFactory = null;
-        }
-    }
-
-    /**
-     * @return new instance of {@link ConnectionFactory}
-     */
-    @Override
-    public ConnectionFactory getConnectionFactory() {
-        if (this.configured) {
-            return this.connectionFactory;
-        }
-        throw new IllegalStateException("ConnectionFactory can not be obtained unless "
-                + "this ControllerService is configured. See onConfigure(ConfigurationContext) method.");
+        return JMSConnectionFactoryProperties.getDynamicPropertyDescriptor(propertyDescriptorName);
     }
 
     @OnEnabled
-    public void enable(ConfigurationContext context) {
-        try {
-            if (!this.configured) {
-                if (logger.isInfoEnabled()) {
-                    logger.info("Configuring " + this.getClass().getSimpleName() + " for '"
-                            + context.getProperty(CONNECTION_FACTORY_IMPL).evaluateAttributeExpressions().getValue() + "' to be connected to '"
-                            + context.getProperty(BROKER_URI).evaluateAttributeExpressions().getValue() + "'");
-                }
-
-                this.createConnectionFactoryInstance(context);
-                this.setConnectionFactoryProperties(context);
-            }
-            this.configured = true;
-        } catch (Exception e) {
-            logger.error("Failed to configure " + this.getClass().getSimpleName(), e);
-            this.configured = false;
-            throw new IllegalStateException(e);
-        }
+    public void onEnabled(ConfigurationContext context) {
+        delegate = new JMSConnectionFactoryHandler(context, getLogger());
     }
 
     @OnDisabled
-    public void disable() {
-        this.connectionFactory = null;
-        this.configured = false;
-    }
-
-    /**
-     * This operation follows standard bean convention by matching property name
-     * to its corresponding 'setter' method. Once the method was located it is
-     * invoked to set the corresponding property to a value provided by during
-     * service configuration. For example, 'channel' property will correspond to
-     * 'setChannel(..) method and 'queueManager' property will correspond to
-     * setQueueManager(..) method with a single argument. The bean convention is also
-     * explained in user manual for this component with links pointing to
-     * documentation of various ConnectionFactories.
-     * <p>
-     * There are also few adjustments to accommodate well known brokers. For
-     * example ActiveMQ ConnectionFactory accepts address of the Message Broker
-     * in a form of URL while IBMs in the form of host/port pair(s).
-     * <p>
-     * This method will use the value retrieved from the 'BROKER_URI' static
-     * property as is. An exception to this if ConnectionFactory implementation
-     * is coming from IBM MQ and connecting to a stand-alone queue manager. In
-     * this case the Broker URI is expected to be entered as a colon separated
-     * host/port pair, which then is split on ':' and the resulting pair will be
-     * used to execute setHostName(..) and setPort(..) methods on the provided
-     * ConnectionFactory.
-     * <p>
-     * This method may need to be maintained and adjusted to accommodate other
-     * implementation of ConnectionFactory, but only for URL/Host/Port issue.
-     * All other properties are set as dynamic properties where user essentially
-     * provides both property name and value.
-     *
-     * @see <a href="http://activemq.apache.org/maven/apidocs/org/apache/activemq/ActiveMQConnectionFactory.html#setBrokerURL-java.lang.String-">setBrokerURL(String brokerURL)</a>
-     * @see <a href="https://docs.tibco.com/pub/enterprise_message_service/8.1.0/doc/html/tib_ems_api_reference/api/javadoc/com/tibco/tibjms/TibjmsConnectionFactory.html#setServerUrl(java.lang.String)">setServerUrl(String serverUrl)</a>
-     * @see <a href="https://www.ibm.com/support/knowledgecenter/en/SSFKSJ_7.1.0/com.ibm.mq.javadoc.doc/WMQJMSClasses/com/ibm/mq/jms/MQConnectionFactory.html#setHostName_java.lang.String_">setHostName(String hostname)</a>
-     * @see <a href="https://www.ibm.com/support/knowledgecenter/en/SSFKSJ_7.1.0/com.ibm.mq.javadoc.doc/WMQJMSClasses/com/ibm/mq/jms/MQConnectionFactory.html#setPort_int_">setPort(int port)</a>
-     * @see <a href="https://www.ibm.com/support/knowledgecenter/en/SSFKSJ_7.1.0/com.ibm.mq.javadoc.doc/WMQJMSClasses/com/ibm/mq/jms/MQConnectionFactory.html#setConnectionNameList_java.lang.String_">setConnectionNameList(String hosts)</a>
-     * @see #setProperty(String propertyName, Object propertyValue)
-     */
-    void setConnectionFactoryProperties(ConfigurationContext context) {
-        if (context.getProperty(BROKER_URI).isSet()) {
-            String brokerValue = context.getProperty(BROKER_URI).evaluateAttributeExpressions().getValue();
-            String connectionFactoryValue = context.getProperty(CONNECTION_FACTORY_IMPL).evaluateAttributeExpressions().getValue();
-            if (connectionFactoryValue.startsWith("org.apache.activemq")) {
-                this.setProperty("brokerURL", brokerValue);
-            } else if (connectionFactoryValue.startsWith("com.tibco.tibjms")) {
-                this.setProperty("serverUrl", brokerValue);
-            } else {
-                String[] brokerList = brokerValue.split(",");
-                if (connectionFactoryValue.startsWith("com.ibm.mq.jms")) {
-                    List<String> ibmConList = new ArrayList<String>();
-                    for (String broker : brokerList) {
-                        String[] hostPort = broker.split(":");
-                        if (hostPort.length == 2) {
-                            ibmConList.add(hostPort[0]+"("+hostPort[1]+")");
-                        } else {
-                            ibmConList.add(broker);
-                        }
-                    }
-                    this.setProperty("connectionNameList", String.join(",", ibmConList));
-                } else {
-                    // Try to parse broker URI as colon separated host/port pair. Use first pair if multiple given.
-                    String[] hostPort = brokerList[0].split(":");
-                    if (hostPort.length == 2) {
-                        // If broker URI indeed was colon separated host/port pair
-                        this.setProperty("hostName", hostPort[0]);
-                        this.setProperty("port", hostPort[1]);
-                    }
-                }
-            }
-        }
-
-        SSLContextService sc = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-        if (sc != null) {
-            SSLContext ssl = sc.createSSLContext(ClientAuth.NONE);
-            this.setProperty("sSLSocketFactory", ssl.getSocketFactory());
-        }
-
-        List<Entry<PropertyDescriptor, String>> dynamicProperties = context.getProperties().entrySet().stream()
-                .filter(entry -> entry.getKey().isDynamic())
-                .collect(Collectors.toList());
-
-        for (Entry<PropertyDescriptor, String> entry : dynamicProperties) {
-            PropertyDescriptor descriptor = entry.getKey();
-            String propertyName = descriptor.getName();
-            String propertyValue = context.getProperty(descriptor).evaluateAttributeExpressions().getValue();
-            this.setProperty(propertyName, propertyValue);
-        }
+    public void onDisabled() {
+        delegate = null;
     }
 
-    /**
-     * Sets corresponding {@link ConnectionFactory}'s property to a
-     * 'propertyValue' by invoking a 'setter' method that corresponds to
-     * 'propertyName'. For example, 'channel' property will correspond to
-     * 'setChannel(..) method and 'queueManager' property will correspond to
-     * setQueueManager(..) method with a single argument.
-     * <p>
-     * NOTE: There is a limited type conversion to accommodate property value
-     * types since all NiFi configuration properties comes as String. It is
-     * accomplished by checking the argument type of the method and executing
-     * its corresponding conversion to target primitive (e.g., value 'true' will
-     * go thru Boolean.parseBoolean(propertyValue) if method argument is of type
-     * boolean). None-primitive values are not supported at the moment and will
-     * result in {@link IllegalArgumentException}. It is OK though since based
-     * on analysis of several ConnectionFactory implementation the all seem to
-     * follow bean convention and all their properties using Java primitives as
-     * arguments.
-     */
-    void setProperty(String propertyName, Object propertyValue) {
-        String methodName = this.toMethodName(propertyName);
-        Method[] methods = Utils.findMethods(methodName, this.connectionFactory.getClass());
-        if (methods != null && methods.length > 0) {
-            try {
-                for (Method method : methods) {
-                    Class<?> returnType = method.getParameterTypes()[0];
-                    if (String.class.isAssignableFrom(returnType)) {
-                        method.invoke(this.connectionFactory, propertyValue);
-                        return;
-                    } else if (int.class.isAssignableFrom(returnType)) {
-                        method.invoke(this.connectionFactory, Integer.parseInt((String) propertyValue));
-                        return;
-                    } else if (long.class.isAssignableFrom(returnType)) {
-                        method.invoke(this.connectionFactory, Long.parseLong((String) propertyValue));
-                        return;
-                    } else if (boolean.class.isAssignableFrom(returnType)) {
-                        method.invoke(this.connectionFactory, Boolean.parseBoolean((String) propertyValue));
-                        return;
-                    }
-                }
-                methods[0].invoke(this.connectionFactory, propertyValue);
-            } catch (Exception e) {
-                throw new IllegalStateException("Failed to set property " + propertyName, e);
-            }
-        } else if (propertyName.equals("hostName")) {
-            this.setProperty("host", propertyValue); // try 'host' as another common convention.
-        }
-    }
-
-    /**
-     * Creates an instance of the {@link ConnectionFactory} from the provided
-     * 'CONNECTION_FACTORY_IMPL'.
-     */
-    private void createConnectionFactoryInstance(ConfigurationContext context) {
-        String connectionFactoryImplName = context.getProperty(CONNECTION_FACTORY_IMPL).evaluateAttributeExpressions().getValue();
-        this.connectionFactory = Utils.newDefaultInstance(connectionFactoryImplName);
+    @Override
+    public ConnectionFactory getConnectionFactory() {
+        return delegate.getConnectionFactory();
     }
 
-    /**
-     * Will convert propertyName to a method name following bean convention. For
-     * example, 'channel' property will correspond to 'setChannel method and
-     * 'queueManager' property will correspond to setQueueManager method name
-     */
-    private String toMethodName(String propertyName) {
-        char c[] = propertyName.toCharArray();
-        c[0] = Character.toUpperCase(c[0]);
-        return "set" + new String(c);
+    @Override
+    public void resetConnectionFactory(ConnectionFactory cachedFactory) {
+        delegate.resetConnectionFactory(cachedFactory);
     }
 
-    /**
-     * {@link Validator} that ensures that brokerURI's length > 0 after EL
-     * evaluation
-     */
-    static class NonEmptyBrokerURIValidator implements Validator {
-        @Override
-        public ValidationResult validate(String subject, String input, ValidationContext context) {
-            if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input)) {
-                return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build();
-            }
-            return StandardValidators.NON_EMPTY_VALIDATOR.validate(subject, input, context);
-        }
-    }
 }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryHandler.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryHandler.java
new file mode 100644
index 0000000..53c2163
--- /dev/null
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryHandler.java
@@ -0,0 +1,154 @@
+/*
+ * 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.jms.cf;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import javax.jms.ConnectionFactory;
+import javax.naming.Context;
+import javax.naming.InitialContext;
+import javax.naming.NamingException;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Hashtable;
+import java.util.Set;
+
+import static org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME;
+import static org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties.JNDI_CREDENTIALS;
+import static org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY;
+import static org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties.JNDI_PRINCIPAL;
+import static org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties.JNDI_PROVIDER_URL;
+
+/**
+ * Handler class to retrieve a JMS Connection Factory object via JNDI.
+ * The handler can be used from controller services and processors as well.
+ */
+public class JndiJmsConnectionFactoryHandler implements IJMSConnectionFactoryProvider {
+
+    private final PropertyContext context;
+    private final Set<PropertyDescriptor> propertyDescriptors;
+    private final ComponentLog logger;
+
+    private volatile ConnectionFactory connectionFactory;
+
+    public JndiJmsConnectionFactoryHandler(ConfigurationContext context, ComponentLog logger) {
+        this.context = context;
+        this.propertyDescriptors = context.getProperties().keySet();
+        this.logger = logger;
+    }
+
+    public JndiJmsConnectionFactoryHandler(ProcessContext context, ComponentLog logger) {
+        this.context = context;
+        this.propertyDescriptors = context.getProperties().keySet();
+        this.logger = logger;
+    }
+
+    @Override
+    public synchronized ConnectionFactory getConnectionFactory() {
+        if (connectionFactory == null) {
+            connectionFactory = lookupConnectionFactory();
+        } else {
+            logger.debug("Connection Factory has already been obtained from JNDI. Will return cached instance.");
+        }
+
+        return connectionFactory;
+    }
+
+    @Override
+    public synchronized void resetConnectionFactory(ConnectionFactory cachedFactory) {
+        if (cachedFactory == connectionFactory) {
+            logger.debug("Resetting connection factory");
+            connectionFactory = null;
+        }
+    }
+
+    private ConnectionFactory lookupConnectionFactory() {
+        try {
+            final String factoryName = context.getProperty(JNDI_CONNECTION_FACTORY_NAME).evaluateAttributeExpressions().getValue().trim();
+            logger.debug("Looking up Connection Factory with name [{}]", new Object[] {factoryName});
+
+            final Context initialContext = createInitialContext();
+            final Object factoryObject = initialContext.lookup(factoryName);
+
+            logger.debug("Obtained {} from JNDI", new Object[] {factoryObject});
+
+            if (factoryObject == null) {
+                throw new ProcessException("Got a null Factory Object from JNDI");
+            }
+            if (!(factoryObject instanceof ConnectionFactory)) {
+                throw new ProcessException("Successfully performed JNDI lookup with Object Name [" + factoryName + "] but the returned object is not a ConnectionFactory. " +
+                    "Instead, is of type " + factoryObject.getClass() + " : " + factoryObject);
+            }
+
+            return (ConnectionFactory) instrumentWithClassLoader(factoryObject, Thread.currentThread().getContextClassLoader(), ConnectionFactory.class);
+        } catch (final NamingException ne) {
+            throw new ProcessException("Could not obtain JMS Connection Factory from JNDI", ne);
+        }
+    }
+
+
+    private Context createInitialContext() throws NamingException {
+        final Hashtable<String, String> env = new Hashtable<>();
+        env.put(Context.INITIAL_CONTEXT_FACTORY, context.getProperty(JNDI_INITIAL_CONTEXT_FACTORY).evaluateAttributeExpressions().getValue().trim());
+        env.put(Context.PROVIDER_URL, context.getProperty(JNDI_PROVIDER_URL).evaluateAttributeExpressions().getValue().trim());
+
+        final String principal = context.getProperty(JNDI_PRINCIPAL).evaluateAttributeExpressions().getValue();
+        if (principal != null) {
+            env.put(Context.SECURITY_PRINCIPAL, principal);
+        }
+
+        final String credentials = context.getProperty(JNDI_CREDENTIALS).getValue();
+        if (credentials != null) {
+            env.put(Context.SECURITY_CREDENTIALS, credentials);
+        }
+
+        propertyDescriptors.forEach(descriptor -> {
+            if (descriptor.isDynamic()) {
+                env.put(descriptor.getName(), context.getProperty(descriptor).evaluateAttributeExpressions().getValue());
+            }
+        });
+
+        logger.debug("Creating Initial Context using JNDI Environment {}", new Object[] {env});
+
+        final Context initialContext = new InitialContext(env);
+        return initialContext;
+    }
+
+    private static Object instrumentWithClassLoader(final Object obj, final ClassLoader classLoader, final Class<?>... interfaces) {
+        final InvocationHandler invocationHandler = new InvocationHandler() {
+            @Override
+            public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
+                final Thread thread = Thread.currentThread();
+                final ClassLoader currentClassLoader = thread.getContextClassLoader();
+                try {
+                    thread.setContextClassLoader(classLoader);
+                    return method.invoke(obj, args);
+                } finally {
+                    thread.setContextClassLoader(currentClassLoader);
+                }
+            }
+        };
+
+        return Proxy.newProxyInstance(classLoader, interfaces, invocationHandler);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProperties.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProperties.java
new file mode 100644
index 0000000..b42d104
--- /dev/null
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProperties.java
@@ -0,0 +1,114 @@
+/*
+ * 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.jms.cf;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+
+public class JndiJmsConnectionFactoryProperties {
+
+    public static final PropertyDescriptor JNDI_INITIAL_CONTEXT_FACTORY = new Builder()
+            .name("java.naming.factory.initial")
+            .displayName("JNDI Initial Context Factory Class")
+            .description("The fully qualified class name of the JNDI Initial Context Factory Class (java.naming.factory.initial).")
+            .required(true)
+            .addValidator(NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor JNDI_PROVIDER_URL = new Builder()
+            .name("java.naming.provider.url")
+            .displayName("JNDI Provider URL")
+            .description("The URL of the JNDI Provider to use (java.naming.provider.url).")
+            .required(true)
+            .addValidator(NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor JNDI_CONNECTION_FACTORY_NAME = new Builder()
+            .name("connection.factory.name")
+            .displayName("JNDI Name of the Connection Factory")
+            .description("The name of the JNDI Object to lookup for the Connection Factory.")
+            .required(true)
+            .addValidator(NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor JNDI_CLIENT_LIBRARIES = new Builder()
+            .name("naming.factory.libraries")
+            .displayName("JNDI / JMS Client Libraries")
+            .description("Specifies jar files and/or directories (defined as a comma separated list) to add to the ClassPath " +
+                    "in order to load the JNDI / JMS client libraries.")
+            .required(false)
+            .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .dynamicallyModifiesClasspath(true)
+            .build();
+
+    public static final PropertyDescriptor JNDI_PRINCIPAL = new Builder()
+            .name("java.naming.security.principal")
+            .displayName("JNDI Principal")
+            .description("The Principal to use when authenticating with JNDI (java.naming.security.principal).")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor JNDI_CREDENTIALS = new Builder()
+            .name("java.naming.security.credentials")
+            .displayName("JNDI Credentials")
+            .description("The Credentials to use when authenticating with JNDI (java.naming.security.credentials).")
+            .required(false)
+            .addValidator(Validator.VALID)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .sensitive(true)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Arrays.asList(
+            JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY,
+            JndiJmsConnectionFactoryProperties.JNDI_PROVIDER_URL,
+            JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME,
+            JndiJmsConnectionFactoryProperties.JNDI_CLIENT_LIBRARIES,
+            JndiJmsConnectionFactoryProperties.JNDI_PRINCIPAL,
+            JndiJmsConnectionFactoryProperties.JNDI_CREDENTIALS
+    );
+
+    public static List<PropertyDescriptor> getPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    public static PropertyDescriptor getDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new Builder()
+                .name(propertyDescriptorName)
+                .displayName(propertyDescriptorName)
+                .description("JNDI Initial Context Environment configuration for '" + propertyDescriptorName + "'")
+                .required(false)
+                .dynamic(true)
+                .addValidator(Validator.VALID)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+                .build();
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java
index 44d8d99..c14f53d 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java
@@ -21,214 +21,55 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyDescriptor.Builder;
-import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.util.StandardValidators;
 
 import javax.jms.ConnectionFactory;
-import javax.naming.Context;
-import javax.naming.InitialContext;
-import javax.naming.NamingException;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.util.Arrays;
-import java.util.Hashtable;
 import java.util.List;
 
-import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
-
 @Tags({"jms", "jndi", "messaging", "integration", "queue", "topic", "publish", "subscribe"})
 @CapabilityDescription("Provides a service to lookup an existing JMS ConnectionFactory using the Java Naming and Directory Interface (JNDI).")
 @DynamicProperty(
     description = "In order to perform a JNDI Lookup, an Initial Context must be established. When this is done, an Environment can be established for the context. Any dynamic/user-defined property" +
         " that is added to this Controller Service will be added as an Environment configuration/variable to this Context.",
     name = "The name of a JNDI Initial Context environment variable.",
-    value = "The value of the JNDI Initial Context Environment variable.",
+    value = "The value of the JNDI Initial Context environment variable.",
     expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
 @SeeAlso(classNames = {"org.apache.nifi.jms.processors.ConsumeJMS", "org.apache.nifi.jms.processors.PublishJMS", "org.apache.nifi.jms.cf.JMSConnectionFactoryProvider"})
 public class JndiJmsConnectionFactoryProvider extends AbstractControllerService implements JMSConnectionFactoryProviderDefinition {
 
-    static final PropertyDescriptor INITIAL_NAMING_FACTORY_CLASS = new Builder()
-        .name("java.naming.factory.initial")
-        .displayName("Initial Naming Factory Class")
-        .description("The fully qualified class name of the Java Initial Naming Factory (java.naming.factory.initial).")
-        .addValidator(NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-        .required(true)
-        .build();
-    static final PropertyDescriptor NAMING_PROVIDER_URL = new Builder()
-        .name("java.naming.provider.url")
-        .displayName("Naming Provider URL")
-        .description("The URL of the JNDI Naming Provider to use")
-        .required(true)
-        .addValidator(NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-        .build();
-    static final PropertyDescriptor CONNECTION_FACTORY_NAME = new Builder()
-        .name("connection.factory.name")
-        .displayName("Connection Factory Name")
-        .description("The name of the JNDI Object to lookup for the Connection Factory")
-        .required(true)
-        .addValidator(NON_EMPTY_VALIDATOR)
-        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-        .build();
-    static final PropertyDescriptor NAMING_FACTORY_LIBRARIES = new Builder()
-        .name("naming.factory.libraries")
-        .displayName("Naming Factory Libraries")
-        .description("Specifies .jar files or directories to add to the ClassPath in order to find the Initial Naming Factory Class")
-        .required(false)
-        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-        .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
-        .dynamicallyModifiesClasspath(true)
-        .build();
-    static final PropertyDescriptor PRINCIPAL = new Builder()
-        .name("java.naming.security.principal")
-        .displayName("JNDI Principal")
-        .description("The Principal to use when authenticating with JNDI")
-        .required(false)
-        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .build();
-    static final PropertyDescriptor CREDENTIALS = new Builder()
-        .name("java.naming.security.credentials")
-        .displayName("Credentials")
-        .description("The Credentials to use when authenticating with JNDI")
-        .required(false)
-        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-        .addValidator(Validator.VALID)
-        .sensitive(true)
-        .build();
-
-    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Arrays.asList(
-        INITIAL_NAMING_FACTORY_CLASS,
-        NAMING_PROVIDER_URL,
-        CONNECTION_FACTORY_NAME,
-        NAMING_FACTORY_LIBRARIES,
-        PRINCIPAL,
-        CREDENTIALS);
-
-    private ConnectionFactory connectionFactory;
+    private volatile JndiJmsConnectionFactoryHandler delegate;
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return PROPERTY_DESCRIPTORS;
+        return JndiJmsConnectionFactoryProperties.getPropertyDescriptors();
     }
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new Builder()
-            .name(propertyDescriptorName)
-            .displayName(propertyDescriptorName)
-            .description("JNDI Initial Context Environment configuration for '" + propertyDescriptorName + "'")
-            .required(false)
-            .dynamic(true)
-            .addValidator(Validator.VALID)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-            .build();
+        return JndiJmsConnectionFactoryProperties.getDynamicPropertyDescriptor(propertyDescriptorName);
     }
 
-    @OnDisabled
-    public void shutdown() {
-        connectionFactory = null;
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        delegate = new JndiJmsConnectionFactoryHandler(context, getLogger());
     }
 
-    @Override
-    public synchronized void resetConnectionFactory(ConnectionFactory cachedFactory) {
-        if (cachedFactory == connectionFactory) {
-            getLogger().debug("Resetting connection factory");
-            connectionFactory = null;
-        }
+    @OnDisabled
+    public void onDisabled() {
+        delegate = null;
     }
 
     @Override
-    public synchronized ConnectionFactory getConnectionFactory() {
-        if (connectionFactory == null) {
-            connectionFactory = lookupConnectionFactory();
-        } else {
-            getLogger().debug("Connection Factory has already been obtained from JNDI. Will return cached instance.");
-        }
-
-        return connectionFactory;
-    }
-
-
-    private ConnectionFactory lookupConnectionFactory() {
-        try {
-            final ConfigurationContext context = getConfigurationContext();
-
-            final String factoryName = context.getProperty(CONNECTION_FACTORY_NAME).evaluateAttributeExpressions().getValue().trim();
-            getLogger().debug("Looking up Connection Factory with name [{}]", new Object[] {factoryName});
-
-            final Context initialContext = createInitialContext();
-            final Object factoryObject = initialContext.lookup(factoryName);
-
-            getLogger().debug("Obtained {} from JNDI", new Object[] {factoryObject});
-
-            if (factoryObject == null) {
-                throw new ProcessException("Got a null Factory Object from JNDI");
-            }
-            if (!(factoryObject instanceof ConnectionFactory)) {
-                throw new ProcessException("Successfully performed JNDI lookup with Object Name [" + factoryName + "] but the returned object is not a ConnectionFactory. " +
-                    "Instead, is of type " + factoryObject.getClass() + " : " + factoryObject);
-            }
-
-            return (ConnectionFactory) instrumentWithClassLoader(factoryObject, Thread.currentThread().getContextClassLoader(), ConnectionFactory.class);
-        } catch (final NamingException ne) {
-            throw new ProcessException("Could not obtain JMS Connection Factory from JNDI", ne);
-        }
+    public ConnectionFactory getConnectionFactory() {
+        return delegate.getConnectionFactory();
     }
 
-
-    private Context createInitialContext() throws NamingException {
-        final ConfigurationContext context = getConfigurationContext();
-
-        final Hashtable<String, String> env = new Hashtable<>();
-        env.put(Context.INITIAL_CONTEXT_FACTORY, context.getProperty(INITIAL_NAMING_FACTORY_CLASS).evaluateAttributeExpressions().getValue().trim());
-        env.put(Context.PROVIDER_URL, context.getProperty(NAMING_PROVIDER_URL).evaluateAttributeExpressions().getValue().trim());
-
-        final String principal = context.getProperty(PRINCIPAL).evaluateAttributeExpressions().getValue();
-        if (principal != null) {
-            env.put(Context.SECURITY_PRINCIPAL, principal);
-        }
-
-        final String credentials = context.getProperty(CREDENTIALS).getValue();
-        if (credentials != null) {
-            env.put(Context.SECURITY_CREDENTIALS, credentials);
-        }
-
-        context.getProperties().keySet().forEach(descriptor -> {
-            if (descriptor.isDynamic()) {
-                env.put(descriptor.getName(), context.getProperty(descriptor).evaluateAttributeExpressions().getValue());
-            }
-        });
-
-        getLogger().debug("Creating Initial Context using JNDI Environment {}", new Object[] {env});
-
-        final Context initialContext = new InitialContext(env);
-        return initialContext;
-    }
-
-    public static Object instrumentWithClassLoader(final Object obj, final ClassLoader classLoader, final Class<?>... interfaces) {
-        final InvocationHandler invocationHandler = new InvocationHandler() {
-            @Override
-            public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
-                final Thread thread = Thread.currentThread();
-                final ClassLoader currentClassLoader = thread.getContextClassLoader();
-                try {
-                    thread.setContextClassLoader(classLoader);
-                    return method.invoke(obj, args);
-                } finally {
-                    thread.setContextClassLoader(currentClassLoader);
-                }
-            }
-        };
-
-        return Proxy.newProxyInstance(classLoader, interfaces, invocationHandler);
+    @Override
+    public void resetConnectionFactory(ConnectionFactory cachedFactory) {
+        delegate.resetConnectionFactory(cachedFactory);
     }
 }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index 74f826c..39b8dac 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -18,10 +18,19 @@ package org.apache.nifi.jms.processors;
 
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.jms.cf.IJMSConnectionFactoryProvider;
+import org.apache.nifi.jms.cf.JMSConnectionFactoryHandler;
+import org.apache.nifi.jms.cf.JMSConnectionFactoryProperties;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProviderDefinition;
+import org.apache.nifi.jms.cf.JndiJmsConnectionFactoryHandler;
+import org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -37,10 +46,13 @@ import javax.jms.ConnectionFactory;
 import javax.jms.Message;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
 
 /**
  * Base JMS processor to support implementation of JMS producers and consumers.
@@ -102,15 +114,7 @@ public abstract class AbstractJMSProcessor<T extends JMSWorker> extends Abstract
             .defaultValue("1")
             .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
             .build();
-    static final PropertyDescriptor MESSAGE_BODY = new PropertyDescriptor.Builder()
-            .name("message-body-type")
-            .displayName("Message Body Type")
-            .description("The type of JMS message body to construct.")
-            .required(true)
-            .defaultValue(BYTES_MESSAGE)
-            .allowableValues(BYTES_MESSAGE, TEXT_MESSAGE)
-            .build();
-    public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
+    static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
             .name("character-set")
             .displayName("Character Set")
             .description("The name of the character set to use to construct or interpret TextMessages")
@@ -119,61 +123,57 @@ public abstract class AbstractJMSProcessor<T extends JMSWorker> extends Abstract
             .defaultValue(Charset.defaultCharset().name())
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
-    static final PropertyDescriptor ALLOW_ILLEGAL_HEADER_CHARS = new PropertyDescriptor.Builder()
-            .name("allow-illegal-chars-in-jms-header-names")
-            .displayName("Allow Illegal Characters in Header Names")
-            .description("Specifies whether illegal characters in header names should be sent to the JMS broker. " +
-                    "Usually hyphens and full-stops.")
-            .required(true)
-            .defaultValue("false")
-            .allowableValues("true", "false")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-    public static final PropertyDescriptor ATTRIBUTES_AS_HEADERS_REGEX = new PropertyDescriptor.Builder()
-            .name("attributes-to-send-as-jms-headers-regex")
-            .displayName("Attributes to Send as JMS Headers (Regex)")
-            .description("Specifies the Regular Expression that determines the names of FlowFile attributes that" +
-                    " should be sent as JMS Headers")
-            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-            .defaultValue(".*")
-            .required(true)
-            .build();
-
 
     static final PropertyDescriptor CF_SERVICE = new PropertyDescriptor.Builder()
             .name("Connection Factory Service")
-            .description("The Controller Service that is used to obtain ConnectionFactory")
-            .required(true)
+            .description("The Controller Service that is used to obtain Connection Factory. Alternatively, the 'JNDI *' or the 'JMS *' properties " +
+                    "can also be be used to configure the Connection Factory.")
+            .required(false)
             .identifiesControllerService(JMSConnectionFactoryProviderDefinition.class)
             .build();
 
-    static final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>();
+    static final List<PropertyDescriptor> JNDI_JMS_CF_PROPERTIES = Collections.unmodifiableList(
+            JndiJmsConnectionFactoryProperties.getPropertyDescriptors().stream()
+                    .map(pd -> new PropertyDescriptor.Builder()
+                            .fromPropertyDescriptor(pd)
+                            .required(false)
+                            .build())
+                    .collect(Collectors.toList())
+    );
+
+    static final List<PropertyDescriptor> JMS_CF_PROPERTIES = Collections.unmodifiableList(
+            JMSConnectionFactoryProperties.getPropertyDescriptors().stream()
+                    .map(pd -> new PropertyDescriptor.Builder()
+                            .fromPropertyDescriptor(pd)
+                            .required(false)
+                            .build())
+                    .collect(Collectors.toList())
+    );
+
+
+    private volatile IJMSConnectionFactoryProvider connectionFactoryProvider;
     private volatile BlockingQueue<T> workerPool;
     private final AtomicInteger clientIdCounter = new AtomicInteger(1);
 
-    static {
-        propertyDescriptors.add(CF_SERVICE);
-        propertyDescriptors.add(DESTINATION);
-        propertyDescriptors.add(DESTINATION_TYPE);
-        propertyDescriptors.add(USER);
-        propertyDescriptors.add(PASSWORD);
-        propertyDescriptors.add(CLIENT_ID);
-        propertyDescriptors.add(SESSION_CACHE_SIZE);
-        propertyDescriptors.add(MESSAGE_BODY);
-        propertyDescriptors.add(CHARSET);
-        propertyDescriptors.add(ALLOW_ILLEGAL_HEADER_CHARS);
-        propertyDescriptors.add(ATTRIBUTES_AS_HEADERS_REGEX);
-    }
-
     protected static String getClientId(ProcessContext context) {
         return context.getProperty(CLIENT_ID).evaluateAttributeExpressions().getValue();
-      }
+    }
 
     @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return propertyDescriptors;
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .description("Additional configuration property for the Connection Factory")
+                .name(propertyDescriptorName)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+                .dynamic(true)
+                .build();
     }
 
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        return new ConnectionFactoryConfigValidator(validationContext).validateConnectionFactoryConfig();
+    }
 
     @Override
     public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
@@ -194,17 +194,16 @@ public abstract class AbstractJMSProcessor<T extends JMSWorker> extends Abstract
             //and reconnection is required.
             if (worker == null || !worker.isValid()){
                 getLogger().debug("Worker is invalid. Will try re-create... ");
-                final JMSConnectionFactoryProviderDefinition cfProvider = context.getProperty(CF_SERVICE).asControllerService(JMSConnectionFactoryProviderDefinition.class);
                 try {
                     if (worker != null) {
                         worker.shutdown();
                     }
                     // Safe to cast. Method #buildTargetResource(ProcessContext context) sets only CachingConnectionFactory
                     CachingConnectionFactory currentCF = (CachingConnectionFactory)worker.jmsTemplate.getConnectionFactory();
-                    cfProvider.resetConnectionFactory(currentCF.getTargetConnectionFactory());
+                    connectionFactoryProvider.resetConnectionFactory(currentCF.getTargetConnectionFactory());
                     worker = buildTargetResource(context);
                 }catch(Exception e) {
-                    getLogger().error("Failed to rebuild:  " + cfProvider);
+                    getLogger().error("Failed to rebuild:  " + connectionFactoryProvider);
                     worker = null;
                 }
             }
@@ -219,6 +218,24 @@ public abstract class AbstractJMSProcessor<T extends JMSWorker> extends Abstract
     }
 
     @OnScheduled
+    public void setupConnectionFactoryProvider(final ProcessContext context) {
+        if (context.getProperty(CF_SERVICE).isSet()) {
+            connectionFactoryProvider = context.getProperty(CF_SERVICE).asControllerService(JMSConnectionFactoryProviderDefinition.class);
+        } else if (context.getProperty(JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME).isSet()) {
+            connectionFactoryProvider = new JndiJmsConnectionFactoryHandler(context, getLogger());
+        } else if (context.getProperty(JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL).isSet()) {
+            connectionFactoryProvider = new JMSConnectionFactoryHandler(context, getLogger());
+        } else {
+            throw new ProcessException("No Connection Factory configured.");
+        }
+    }
+
+    @OnUnscheduled
+    public void shutdownConnectionFactoryProvider(final ProcessContext context) {
+        connectionFactoryProvider = null;
+    }
+
+    @OnScheduled
     public void setupWorkerPool(final ProcessContext context) {
         workerPool = new LinkedBlockingQueue<>(context.getMaxConcurrentTasks());
     }
@@ -257,8 +274,7 @@ public abstract class AbstractJMSProcessor<T extends JMSWorker> extends Abstract
      * {@link JmsTemplate} used by this Processor.
      */
     private T buildTargetResource(ProcessContext context) {
-        final JMSConnectionFactoryProviderDefinition cfProvider = context.getProperty(CF_SERVICE).asControllerService(JMSConnectionFactoryProviderDefinition.class);
-        final ConnectionFactory connectionFactory = cfProvider.getConnectionFactory();
+        final ConnectionFactory connectionFactory = connectionFactoryProvider.getConnectionFactory();
 
         final UserCredentialsConnectionFactoryAdapter cfCredentialsAdapter = new UserCredentialsConnectionFactoryAdapter();
         cfCredentialsAdapter.setTargetConnectionFactory(connectionFactory);
@@ -291,4 +307,93 @@ public abstract class AbstractJMSProcessor<T extends JMSWorker> extends Abstract
         }
     }
 
+    static class ConnectionFactoryConfigValidator {
+
+        private final ValidationContext validationContext;
+
+        private final PropertyValue connectionFactoryServiceProperty;
+        private final PropertyValue jndiInitialContextFactoryProperty;
+        private final PropertyValue jmsConnectionFactoryImplProperty;
+
+        ConnectionFactoryConfigValidator(ValidationContext validationContext) {
+            this.validationContext = validationContext;
+
+            connectionFactoryServiceProperty = validationContext.getProperty(CF_SERVICE);
+            jndiInitialContextFactoryProperty = validationContext.getProperty(JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY);
+            jmsConnectionFactoryImplProperty = validationContext.getProperty(JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL);
+        }
+
+        List<ValidationResult> validateConnectionFactoryConfig() {
+            List<ValidationResult> results = new ArrayList<>();
+
+            if (!(connectionFactoryServiceProperty.isSet() || jndiInitialContextFactoryProperty.isSet() || jmsConnectionFactoryImplProperty.isSet())) {
+                results.add(new ValidationResult.Builder()
+                        .subject("Connection Factory config")
+                        .valid(false)
+                        .explanation(String.format("either '%s', '%s' or '%s' must be specified.", CF_SERVICE.getDisplayName(),
+                                JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY.getDisplayName(), JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL.getDisplayName()))
+                        .build());
+            } else if (connectionFactoryServiceProperty.isSet()) {
+                if (hasLocalJndiJmsConnectionFactoryConfig()) {
+                    results.add(new ValidationResult.Builder()
+                            .subject("Connection Factory config")
+                            .valid(false)
+                            .explanation(String.format("cannot set both '%s' and 'JNDI *' properties.", CF_SERVICE.getDisplayName()))
+                            .build());
+                }
+                if (hasLocalJMSConnectionFactoryConfig()) {
+                    results.add(new ValidationResult.Builder()
+                            .subject("Connection Factory config")
+                            .valid(false)
+                            .explanation(String.format("cannot set both '%s' and 'JMS *' properties.", CF_SERVICE.getDisplayName()))
+                            .build());
+                }
+            } else if (hasLocalJndiJmsConnectionFactoryConfig() && hasLocalJMSConnectionFactoryConfig()) {
+                results.add(new ValidationResult.Builder()
+                        .subject("Connection Factory config")
+                        .valid(false)
+                        .explanation("cannot set both 'JNDI *' and 'JMS *' properties.")
+                        .build());
+            } else if (jndiInitialContextFactoryProperty.isSet()) {
+                validateLocalConnectionFactoryConfig(JndiJmsConnectionFactoryProperties.getPropertyDescriptors(), JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY, results);
+            } else if (jmsConnectionFactoryImplProperty.isSet()) {
+                validateLocalConnectionFactoryConfig(JMSConnectionFactoryProperties.getPropertyDescriptors(), JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, results);
+            }
+
+            return results;
+        }
+
+        private boolean hasLocalJndiJmsConnectionFactoryConfig() {
+            return hasLocalConnectionFactoryConfig(JndiJmsConnectionFactoryProperties.getPropertyDescriptors());
+        }
+
+        private boolean hasLocalJMSConnectionFactoryConfig() {
+            return hasLocalConnectionFactoryConfig(JMSConnectionFactoryProperties.getPropertyDescriptors());
+        }
+
+        private boolean hasLocalConnectionFactoryConfig(List<PropertyDescriptor> localConnectionFactoryProperties) {
+            for (PropertyDescriptor propertyDescriptor : localConnectionFactoryProperties) {
+                PropertyValue propertyValue = validationContext.getProperty(propertyDescriptor);
+                if (propertyValue.isSet()) {
+                    return true;
+                }
+            }
+            return false;
+        }
+
+        private void validateLocalConnectionFactoryConfig(List<PropertyDescriptor> localConnectionFactoryProperties, PropertyDescriptor indicatorProperty, List<ValidationResult> results) {
+            for (PropertyDescriptor propertyDescriptor : localConnectionFactoryProperties) {
+                if (propertyDescriptor.isRequired()) {
+                    PropertyValue propertyValue = validationContext.getProperty(propertyDescriptor);
+                    if (!propertyValue.isSet()) {
+                        results.add(new ValidationResult.Builder()
+                                .subject("Connection Factory config")
+                                .valid(false)
+                                .explanation(String.format("'%s' must be specified when '%s' has been configured.", propertyDescriptor.getDisplayName(), indicatorProperty.getDisplayName()))
+                                .build());
+                    }
+                }
+            }
+        }
+    }
 }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index 3e278bc..80f9457 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.jms.processors;
 
+import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
@@ -81,6 +82,10 @@ import java.util.concurrent.TimeUnit;
         @WritesAttribute(attribute = ConsumeJMS.JMS_MESSAGETYPE, description = "The JMS message type, can be TextMessage, BytesMessage, ObjectMessage, MapMessage or StreamMessage)."),
         @WritesAttribute(attribute = "other attributes", description = "Each message property is written to an attribute.")
 })
+@DynamicProperty(name = "The name of a Connection Factory configuration property.", value = "The value of a given Connection Factory configuration property.",
+        description = "Additional configuration property for the Connection Factory. It can be used when the Connection Factory is being configured via the 'JNDI *' or the 'JMS *'" +
+                "properties of the processor. For more information, see the Additional Details page.",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
 @SeeAlso(value = { PublishJMS.class, JMSConnectionFactoryProvider.class })
 public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
     public static final String JMS_MESSAGETYPE = "jms.messagetype";
@@ -161,21 +166,25 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
 
     private final static Set<Relationship> relationships;
 
-    private final static List<PropertyDescriptor> thisPropertyDescriptors;
+    private final static List<PropertyDescriptor> propertyDescriptors;
 
     static {
         List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
-        _propertyDescriptors.addAll(propertyDescriptors);
-        _propertyDescriptors.remove(MESSAGE_BODY);
-        _propertyDescriptors.remove(ALLOW_ILLEGAL_HEADER_CHARS);
-        _propertyDescriptors.remove(ATTRIBUTES_AS_HEADERS_REGEX);
+
+        _propertyDescriptors.add(CF_SERVICE);
+        _propertyDescriptors.add(DESTINATION);
+        _propertyDescriptors.add(DESTINATION_TYPE);
+        _propertyDescriptors.add(USER);
+        _propertyDescriptors.add(PASSWORD);
+        _propertyDescriptors.add(CLIENT_ID);
+        _propertyDescriptors.add(SESSION_CACHE_SIZE);
 
         // change the validator on CHARSET property
-        _propertyDescriptors.remove(CHARSET);
-        PropertyDescriptor CHARSET_WITH_EL_VALIDATOR_PROPERTY = new PropertyDescriptor.Builder().fromPropertyDescriptor(CHARSET)
+        PropertyDescriptor charsetWithELValidatorProperty = new PropertyDescriptor.Builder()
+                .fromPropertyDescriptor(CHARSET)
                 .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR_WITH_EVALUATION)
                 .build();
-        _propertyDescriptors.add(CHARSET_WITH_EL_VALIDATOR_PROPERTY);
+        _propertyDescriptors.add(charsetWithELValidatorProperty);
 
         _propertyDescriptors.add(ACKNOWLEDGEMENT_MODE);
         _propertyDescriptors.add(DURABLE_SUBSCRIBER);
@@ -183,7 +192,11 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
         _propertyDescriptors.add(SUBSCRIPTION_NAME);
         _propertyDescriptors.add(TIMEOUT);
         _propertyDescriptors.add(ERROR_QUEUE);
-        thisPropertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+
+        _propertyDescriptors.addAll(JNDI_JMS_CF_PROPERTIES);
+        _propertyDescriptors.addAll(JMS_CF_PROPERTIES);
+
+        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
 
         Set<Relationship> _relationships = new HashSet<>();
         _relationships.add(REL_SUCCESS);
@@ -292,7 +305,7 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return thisPropertyDescriptors;
+        return propertyDescriptors;
     }
 
     /**
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
index 9cb2451..fb82d50 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.jms.processors;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.ReadsAttribute;
@@ -26,6 +27,8 @@ import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
 import org.apache.nifi.processor.ProcessContext;
@@ -33,6 +36,7 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.StreamUtils;
 import org.springframework.jms.connection.CachingConnectionFactory;
 import org.springframework.jms.core.JmsTemplate;
@@ -42,9 +46,11 @@ import javax.jms.Destination;
 import javax.jms.Message;
 import java.io.StringWriter;
 import java.nio.charset.Charset;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
@@ -80,10 +86,42 @@ import java.util.regex.Pattern;
         + " `delay.type` with value `integer` will cause a JMS message property `delay` to be sent as an Integer rather than a String. Supported types are boolean, byte,"
         + " short, integer, long, float, double, and string (which is the default).")
 })
+@DynamicProperty(name = "The name of a Connection Factory configuration property.", value = "The value of a given Connection Factory configuration property.",
+        description = "Additional configuration property for the Connection Factory. It can be used when the Connection Factory is being configured via the 'JNDI *' or the 'JMS *'" +
+                "properties of the processor. For more information, see the Additional Details page.",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
 @SeeAlso(value = { ConsumeJMS.class, JMSConnectionFactoryProvider.class })
 @SystemResourceConsideration(resource = SystemResource.MEMORY)
 public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
 
+    static final PropertyDescriptor MESSAGE_BODY = new PropertyDescriptor.Builder()
+            .name("message-body-type")
+            .displayName("Message Body Type")
+            .description("The type of JMS message body to construct.")
+            .required(true)
+            .defaultValue(BYTES_MESSAGE)
+            .allowableValues(BYTES_MESSAGE, TEXT_MESSAGE)
+            .build();
+    static final PropertyDescriptor ALLOW_ILLEGAL_HEADER_CHARS = new PropertyDescriptor.Builder()
+            .name("allow-illegal-chars-in-jms-header-names")
+            .displayName("Allow Illegal Characters in Header Names")
+            .description("Specifies whether illegal characters in header names should be sent to the JMS broker. " +
+                    "Usually hyphens and full-stops.")
+            .required(true)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    static final PropertyDescriptor ATTRIBUTES_AS_HEADERS_REGEX = new PropertyDescriptor.Builder()
+            .name("attributes-to-send-as-jms-headers-regex")
+            .displayName("Attributes to Send as JMS Headers (Regex)")
+            .description("Specifies the Regular Expression that determines the names of FlowFile attributes that" +
+                    " should be sent as JMS Headers")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .defaultValue(".*")
+            .required(true)
+            .build();
+
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
             .description("All FlowFiles that are sent to the JMS destination are routed to this relationship")
@@ -93,6 +131,7 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
             .description("All FlowFiles that cannot be sent to JMS destination are routed to this relationship")
             .build();
 
+    private static final List<PropertyDescriptor> propertyDescriptors;
     private final static Set<Relationship> relationships;
 
     /*
@@ -100,6 +139,27 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
      * Will also create a Set of relationships
      */
     static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+
+        _propertyDescriptors.add(CF_SERVICE);
+        _propertyDescriptors.add(DESTINATION);
+        _propertyDescriptors.add(DESTINATION_TYPE);
+        _propertyDescriptors.add(USER);
+        _propertyDescriptors.add(PASSWORD);
+        _propertyDescriptors.add(CLIENT_ID);
+        _propertyDescriptors.add(SESSION_CACHE_SIZE);
+
+        _propertyDescriptors.add(MESSAGE_BODY);
+        _propertyDescriptors.add(CHARSET);
+        _propertyDescriptors.add(ALLOW_ILLEGAL_HEADER_CHARS);
+        _propertyDescriptors.add(ATTRIBUTES_AS_HEADERS_REGEX);
+
+        _propertyDescriptors.addAll(JNDI_JMS_CF_PROPERTIES);
+        _propertyDescriptors.addAll(JMS_CF_PROPERTIES);
+
+        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+
+
         Set<Relationship> _relationships = new HashSet<>();
         _relationships.add(REL_SUCCESS);
         _relationships.add(REL_FAILURE);
@@ -169,6 +229,11 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
         }
     }
 
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return propertyDescriptors;
+    }
+
     /**
      *
      */
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JMSConnectionFactoryProvider/additionalDetails.html b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JMSConnectionFactoryProvider/additionalDetails.html
index 0e92a65..ef1ab82 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JMSConnectionFactoryProvider/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JMSConnectionFactoryProvider/additionalDetails.html
@@ -47,7 +47,7 @@
     The mandatory configuration property is:
 </p>
 <ul>
-    <li><b>MQ ConnectionFactory Implementation</b> - A fully qualified name of the JMS <i>ConnectionFactory</i>
+    <li><b>JMS Connection Factory Implementation</b> - The fully qualified name of the JMS <i>ConnectionFactory</i>
         implementation class. For example:
         <ul>
             <li>Apache ActiveMQ - <a href="http://activemq.apache.org/maven/5.15.9/apidocs/org/apache/activemq/ActiveMQConnectionFactory.html" target="_blank">org.apache.activemq.ActiveMQConnectionFactory</a></li>
@@ -60,12 +60,11 @@
     The following static configuration properties are optional but required in many cases:
 </p>
 <ul>
-    <li><b>MQ Client Libraries path</b> - Path to the directory with additional resources (i.e. JARs,
-        configuration files, etc.) to be added to the classpath. Such resources typically represent target client
-        libraries for the <i>ConnectionFactory</i> implementation. It is optional if you are using Apache ActiveMQ since
-        its libraries are included with this component.
+    <li><b>JMS Client Libraries</b> - Path to the directory with additional resources (eg. JARs,
+        configuration files, etc.) to be added to the classpath (defined as a comma separated list of values). Such resources typically represent target JMS client
+        libraries for the <i>ConnectionFactory</i> implementation.
     </li>
-    <li><b>Broker URI</b> - URI pointing to the network location of the JMS Message broker. For example:
+    <li><b>JMS Broker URI</b> - URI pointing to the network location of the JMS Message broker. For example:
         <ul>
             <li>Apache ActiveMQ - <i>tcp://myhost:1234</i> for single broker and
                 <i>failover:(tcp://myhost01:1234,tcp://myhost02:1234)</i> for multiple brokers.
@@ -108,26 +107,26 @@
         <th>Comments</th>
     </tr>
     <tr>
-        <td>MQ ConnectionFactory Implementation</td>
+        <td>JMS Connection Factory Implementation</td>
         <td>com.ibm.mq.jms.MQQueueConnectionFactory</td>
         <td>Static</td>
         <td>Vendor provided implementation of QueueConnectionFactory</td>
     </tr>
     <tr>
-        <td>MQ Client Libraries path (i.e. /usr/jms/lib)</td>
+        <td>JMS Client Libraries</td>
         <td>/opt/mqm/java/lib</td>
         <td>Static</td>
         <td>Default installation path of client JAR files on Linux systems</td>
     </tr>
     <tr>
-        <td>Broker URI</td>
+        <td>JMS Broker URI</td>
         <td>mqhost01(1414),mqhost02(1414)</td>
         <td>Static</td>
         <td><a href="https://www.ibm.com/support/knowledgecenter/ro/SSAW57_9.0.0/com.ibm.websphere.nd.multiplatform.doc/ae/ucli_pqcfm.html#MQTopicConnectionFactory_enterporthostname" target="_blank">Connection Name List syntax</a>.
             Colon separated host/port pair(s) is also supported</td>
     </tr>
     <tr>
-        <td>SSL Context Service</td>
+        <td>JMS SSL Context Service</td>
         <td></td>
         <td>Static</td>
         <td>Only required if using SSL/TLS</td>
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProvider/additionalDetails.html b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProvider/additionalDetails.html
index b30720b..eeff65d 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProvider/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProvider/additionalDetails.html
@@ -34,18 +34,19 @@
 <pre>
 <code>
 Hashtable env = new Hashtable();
-env.put(Context.INITIAL_CONTEXT_FACTORY, INITIAL_NAMING_FACTORY_CLASS); // Value for this comes from the "Initial Naming Factory Class" property.
-env.put(Context.PROVIDER_URL, NAMING_PROVIDER_URL); // Value for this comes from the "Naming Provider URL" property.
+env.put(Context.INITIAL_CONTEXT_FACTORY, JNDI_INITIAL_CONTEXT_FACTORY); // Value for this comes from the "JNDI Initial Context Factory Class" property.
+env.put(Context.PROVIDER_URL, JNDI_PROVIDER_URL); // Value for this comes from the "JNDI Provider URL" property.
 env.put("My-Environment-Variable", "Environment-Variable-Value"); // This is accomplished by added a user-defined property with name "My-Environment-Variable" and value "Environment-Variable-Value"
 
 Context initialContext = new InitialContext(env);
-ConnectionFactory connectionFactory = initialContext.lookup(CONNECTION_FACTORY_NAME); // Value for Connection Factory name comes from "Connection Factory Name" property
+ConnectionFactory connectionFactory = initialContext.lookup(JNDI_CONNECTION_FACTORY_NAME); // Value for this comes from the "JNDI Name of the Connection Factory" property
 </code>
 </pre>
 
 <p>
-    It is also important to note that, in order for this to work, the class named by the Initial Naming Factory Class must be available on the classpath.
-    In NiFi, this is accomplished by setting the "Naming Factory Libraries" property to point to one or more .jar files or directories (comma-separated values).
+    It is also important to note that, in order for this to work, the class named by the "JNDI Initial Context Factory Class" must be available on the classpath.
+    The JMS provider specific client classes (like the class of the Connection Factory object to be retrieved from JNDI) must also be available on the classpath.
+    In NiFi, this is accomplished by setting the "JNDI / JMS Client Libraries" property to point to one or more .jar files or directories (comma-separated values).
 </p>
 
 <p>
@@ -67,27 +68,27 @@ ConnectionFactory connectionFactory = initialContext.lookup(CONNECTION_FACTORY_N
     </thead>
     <tbody>
       <tr>
-          <td>Initial Naming Factory Class</td>
+          <td>JNDI Initial Context Factory Class</td>
           <td>org.apache.activemq.jndi.ActiveMQInitialContextFactory</td>
       </tr>
       <tr>
-          <td>Naming Provider URL</td>
+          <td>JNDI Provider URL</td>
           <td>tcp://jms-broker:61616</td>
       </tr>
       <tr>
-          <td>Connection Factory Name</td>
+          <td>JNDI Name of the Connection Factory</td>
           <td>ConnectionFactory</td>
       </tr>
       <tr>
-          <td>Naming Factory Libraries</td>
+          <td>JNDI / JMS Client Libraries</td>
           <td>/opt/apache-activemq-5.15.2/lib/</td>
       </tr>
     </tbody>
 </table>
 
 <p>
-    The above example assumes that there exists as host that is accessible with hostname "jms-broker" and that is running Apache ActiveMQ on port 61616 and also that
-    the jar containing the org.apache.activemq.jndi.ActiveMQInitialContextFactory class can be found within the /opt/apache-activemq-5.15.2/lib/ directory.
+    The above example assumes that there exists a host that is accessible with hostname "jms-broker" and that is running Apache ActiveMQ on port 61616 and also that
+    the jar(s) containing the org.apache.activemq.jndi.ActiveMQInitialContextFactory class and the other JMS client classes can be found within the /opt/apache-activemq-5.15.2/lib/ directory.
 </p>
 
 </body>
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.ConsumeJMS/additionalDetails.html b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.ConsumeJMS/additionalDetails.html
index 8fcc0d7..8d1f92a 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.ConsumeJMS/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.ConsumeJMS/additionalDetails.html
@@ -49,9 +49,30 @@
     <li><b>Destination Type</b> - [REQUIRED] the type of the <i>javax.jms.Destination</i>. Could be one of 'QUEUE' or 'TOPIC'
     Usually provided by the administrator. Defaults to 'QUEUE'.
     </li>
-    <li><b>Connection Factory Service</b> - [REQUIRED] link to a pre-configured instance of org.apache.nifi.jms.cf.JMSConnectionFactoryProvider.
-    </li>
 </ol>
+<h3>Connection Factory Configuration</h3>
+There are multiple ways to configure the Connection Factory for the processor:
+<ul>
+    <li><b>Connection Factory Service</b> property - link to a pre-configured controller service (<i>JndiJmsConnectionFactoryProvider</i> or <i>JMSConnectionFactoryProvider</i>)
+    </li>
+    <li><b>JNDI *</b> properties - processor level configuration, the properties are the same as the properties of <i>JndiJmsConnectionFactoryProvider</i> controller service,
+        the dynamic properties can also be used in this case
+    </li>
+    <li><b>JMS *</b> properties - processor level configuration, the properties are the same as the properties of <i>JMSConnectionFactoryProvider</i> controller service,
+        the dynamic properties can also be used in this case
+    </li>
+</ul>
+<p>
+    The preferred way is to use the Connection Factory Service property and a pre-configured controller service. It is also the most convenient method, because it is enough
+    to configure the controller service once and then it can be used in multiple processors.
+</p>
+<p>
+    However, some JMS client libraries may not work with the controller services due to incompatible Java ClassLoader handling between the 3rd party JMS client library and NiFi.
+    Should you encounter <i>java.lang.ClassCastException</i> errors when using the controller services, please try to configure the Connection Factory via the 'JNDI *' or
+    the 'JMS *' and the dynamic properties of the processor.
+    For more details on these properties, see the documentation of the corresponding controller service (<i>JndiJmsConnectionFactoryProvider</i> for 'JNDI *' and
+    <i>JMSConnectionFactoryProvider</i> for 'JMS *').
+</p>
 
 </body>
 </html>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.PublishJMS/additionalDetails.html b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.PublishJMS/additionalDetails.html
index 6bad536..4cb7093 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.PublishJMS/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/resources/docs/org.apache.nifi.jms.processors.PublishJMS/additionalDetails.html
@@ -52,9 +52,30 @@
     <li><b>Destination Type</b> - [REQUIRED] the type of the <i>javax.jms.Destination</i>. Could be one of 'QUEUE' or 'TOPIC'
     Usually provided by the administrator. Defaults to 'QUEUE'.
     </li>
-    <li><b>Connection Factory Service</b> - [REQUIRED] link to a pre-configured instance of org.apache.nifi.jms.cf.JMSConnectionFactoryProvider.
-    </li>
 </ol>
+<h3>Connection Factory Configuration</h3>
+There are multiple ways to configure the Connection Factory for the processor:
+<ul>
+    <li><b>Connection Factory Service</b> property - link to a pre-configured controller service (<i>JndiJmsConnectionFactoryProvider</i> or <i>JMSConnectionFactoryProvider</i>)
+    </li>
+    <li><b>JNDI *</b> properties - processor level configuration, the properties are the same as the properties of <i>JndiJmsConnectionFactoryProvider</i> controller service,
+        the dynamic properties can also be used in this case
+    </li>
+    <li><b>JMS *</b> properties - processor level configuration, the properties are the same as the properties of <i>JMSConnectionFactoryProvider</i> controller service,
+        the dynamic properties can also be used in this case
+    </li>
+</ul>
+<p>
+    The preferred way is to use the Connection Factory Service property and a pre-configured controller service. It is also the most convenient method, because it is enough
+    to configure the controller service once and then it can be used in multiple processors.
+</p>
+<p>
+    However, some JMS client libraries may not work with the controller services due to incompatible Java ClassLoader handling between the 3rd party JMS client library and NiFi.
+    Should you encounter <i>java.lang.ClassCastException</i> errors when using the controller services, please try to configure the Connection Factory via the 'JNDI *' or
+    the 'JMS *' and the dynamic properties of the processor.
+    For more details on these properties, see the documentation of the corresponding controller service (<i>JndiJmsConnectionFactoryProvider</i> for 'JNDI *' and
+    <i>JMSConnectionFactoryProvider</i> for 'JMS *').
+</p>
 
 </body>
 </html>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderForTest.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandlerForTest.java
similarity index 59%
copy from nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderForTest.java
copy to nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandlerForTest.java
index e2df94f..f8e087a 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderForTest.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandlerForTest.java
@@ -16,34 +16,29 @@
  */
 package org.apache.nifi.jms.cf;
 
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.controller.ConfigurationContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.nifi.logging.ComponentLog;
 
 import java.util.HashMap;
 import java.util.Map;
 
 /**
- * Sub-class of {@link JMSConnectionFactoryProvider} only for testing purpose
+ * Sub-class of {@link JMSConnectionFactoryHandler} only for testing purpose
  */
-public class JMSConnectionFactoryProviderForTest extends JMSConnectionFactoryProvider {
-    private static Logger logger = LoggerFactory.getLogger(JMSConnectionFactoryProviderForTest.class);
+public class JMSConnectionFactoryHandlerForTest extends JMSConnectionFactoryHandler {
 
-    private Map<String, Object> setProperties = new HashMap<>();
+    private Map<String, Object> configuredProperties = new HashMap<>();
 
-    @OnEnabled
-    @Override
-    public void enable(ConfigurationContext context) {
-        setConnectionFactoryProperties(context);
+    public JMSConnectionFactoryHandlerForTest(ConfigurationContext context, ComponentLog logger) {
+        super(context, logger);
     }
 
     @Override
     void setProperty(String propertyName, Object propertyValue) {
-        setProperties.put(propertyName, propertyValue);
+        configuredProperties.put(propertyName, propertyValue);
     }
 
-    public Map<String, Object> getSetProperties() {
-        return setProperties;
+    public Map<String, Object> getConfiguredProperties() {
+        return configuredProperties;
     }
 }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderForTest.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderForTest.java
index e2df94f..9e56e9b 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderForTest.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderForTest.java
@@ -18,32 +18,22 @@ package org.apache.nifi.jms.cf;
 
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.controller.ConfigurationContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
 import java.util.Map;
 
 /**
  * Sub-class of {@link JMSConnectionFactoryProvider} only for testing purpose
  */
 public class JMSConnectionFactoryProviderForTest extends JMSConnectionFactoryProvider {
-    private static Logger logger = LoggerFactory.getLogger(JMSConnectionFactoryProviderForTest.class);
-
-    private Map<String, Object> setProperties = new HashMap<>();
 
     @OnEnabled
     @Override
-    public void enable(ConfigurationContext context) {
-        setConnectionFactoryProperties(context);
-    }
-
-    @Override
-    void setProperty(String propertyName, Object propertyValue) {
-        setProperties.put(propertyName, propertyValue);
+    public void onEnabled(ConfigurationContext context) {
+        delegate = new JMSConnectionFactoryHandlerForTest(context, getLogger());
+        delegate.setConnectionFactoryProperties();
     }
 
-    public Map<String, Object> getSetProperties() {
-        return setProperties;
+    public Map<String, Object> getConfiguredProperties() {
+        return ((JMSConnectionFactoryHandlerForTest) delegate).getConfiguredProperties();
     }
 }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java
index 867db0d..c97c402 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java
@@ -17,8 +17,11 @@
 package org.apache.nifi.jms.cf;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockComponentLog;
+import org.apache.nifi.util.MockConfigurationContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
@@ -28,6 +31,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.URISyntaxException;
+import java.util.Collections;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
@@ -84,9 +88,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, "foo");
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TEST_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, "foo");
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.assertNotValid(cfProvider);
     }
@@ -102,9 +106,9 @@ public class JMSConnectionFactoryProviderTest {
         runner.setVariable("broker.uri", SINGLE_TEST_BROKER_WITH_SCHEME_AND_IP);
         runner.setVariable("client.lib", dummyResource);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, "${broker.uri}");
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, "${client.lib}");
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, "${broker.uri}");
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, "${client.lib}");
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -120,9 +124,9 @@ public class JMSConnectionFactoryProviderTest {
         runner.setVariable("broker.uri", SINGLE_TEST_BROKER_WITH_SCHEME_AND_IP);
         runner.setVariable("client.lib", allDummyResources);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, "${broker.uri}");
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, "${client.lib}");
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, "${broker.uri}");
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, "${client.lib}");
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
 
@@ -134,7 +138,14 @@ public class JMSConnectionFactoryProviderTest {
 
     @Test(expected = IllegalStateException.class)
     public void validateGetConnectionFactoryFailureIfServiceNotConfigured() {
-        new JMSConnectionFactoryProvider().getConnectionFactory();
+        JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider() {
+            @Override
+            protected ComponentLog getLogger() {
+                return new MockComponentLog("cfProvider", this);
+            }
+        };
+        cfProvider.onEnabled(new MockConfigurationContext(Collections.emptyMap(), null));
+        cfProvider.getConnectionFactory();
     }
 
     @Test
@@ -144,9 +155,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TEST_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -158,9 +169,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER_WITH_SCHEME);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TEST_BROKER_WITH_SCHEME);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -172,9 +183,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_TEST_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_TEST_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -186,9 +197,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_ACTIVEMQ_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_ACTIVEMQ_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -200,9 +211,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_ACTIVEMQ_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_ACTIVEMQ_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -214,9 +225,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TIBCO_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TIBCO_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -228,9 +239,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_TIBCO_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_TIBCO_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -242,9 +253,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_IBM_MQ_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_IBM_MQ_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -256,9 +267,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_IBM_MQ_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_IBM_MQ_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -270,9 +281,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_IBM_MQ_MIXED_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_IBM_MQ_MIXED_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -284,9 +295,9 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_IBM_MQ_COLON_PAIR_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_IBM_MQ_COLON_PAIR_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.assertValid(cfProvider);
     }
@@ -298,13 +309,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TEST_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("hostName", HOSTNAME, "port", PORT));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("hostName", HOSTNAME, "port", PORT));
     }
 
     @Test
@@ -314,13 +325,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER_WITH_SCHEME);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TEST_BROKER_WITH_SCHEME);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of());
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of());
     }
 
     @Test
@@ -330,13 +341,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_TEST_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_TEST_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("hostName", "myhost01", "port", "1234"));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("hostName", "myhost01", "port", "1234"));
     }
 
     @Test
@@ -346,13 +357,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_ACTIVEMQ_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_ACTIVEMQ_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("brokerURL", SINGLE_ACTIVEMQ_BROKER));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("brokerURL", SINGLE_ACTIVEMQ_BROKER));
     }
 
     @Test
@@ -362,13 +373,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_ACTIVEMQ_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_ACTIVEMQ_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, ACTIVEMQ_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("brokerURL", MULTIPLE_ACTIVEMQ_BROKERS));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("brokerURL", MULTIPLE_ACTIVEMQ_BROKERS));
     }
 
     @Test
@@ -378,13 +389,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TIBCO_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TIBCO_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("serverUrl", SINGLE_TIBCO_BROKER));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("serverUrl", SINGLE_TIBCO_BROKER));
     }
 
     @Test
@@ -394,13 +405,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_TIBCO_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_TIBCO_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TIBCO_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("serverUrl", MULTIPLE_TIBCO_BROKERS));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("serverUrl", MULTIPLE_TIBCO_BROKERS));
     }
 
     @Test
@@ -410,13 +421,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_IBM_MQ_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_IBM_MQ_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("connectionNameList", SINGLE_IBM_MQ_BROKER));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("connectionNameList", SINGLE_IBM_MQ_BROKER));
     }
 
     @Test
@@ -426,13 +437,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_IBM_MQ_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_IBM_MQ_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("connectionNameList", MULTIPLE_IBM_MQ_BROKERS));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("connectionNameList", MULTIPLE_IBM_MQ_BROKERS));
     }
 
     @Test
@@ -442,13 +453,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_IBM_MQ_MIXED_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_IBM_MQ_MIXED_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("connectionNameList", MULTIPLE_IBM_MQ_BROKERS));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("connectionNameList", MULTIPLE_IBM_MQ_BROKERS));
     }
 
     @Test
@@ -458,13 +469,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, MULTIPLE_IBM_MQ_COLON_PAIR_BROKERS);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, MULTIPLE_IBM_MQ_COLON_PAIR_BROKERS);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("connectionNameList", MULTIPLE_IBM_MQ_BROKERS));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("connectionNameList", MULTIPLE_IBM_MQ_BROKERS));
     }
 
     @Test
@@ -474,13 +485,13 @@ public class JMSConnectionFactoryProviderTest {
         JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest();
         runner.addControllerService(controllerServiceId, cfProvider);
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TEST_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, IBM_MQ_CONNECTION_FACTORY_IMPL);
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("connectionNameList", HOSTNAME + "(" + PORT + ")"));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("connectionNameList", HOSTNAME + "(" + PORT + ")"));
     }
 
     @Test
@@ -492,13 +503,13 @@ public class JMSConnectionFactoryProviderTest {
 
         runner.setVariable("test", "dynamicValue");
 
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource);
-        runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, SINGLE_TEST_BROKER);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CLIENT_LIBRARIES, dummyResource);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL);
         runner.setProperty(cfProvider, "dynamicProperty", "${test}");
 
         runner.enableControllerService(cfProvider);
 
-        assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("dynamicProperty", "dynamicValue", "hostName", HOSTNAME, "port", PORT));
+        assertEquals(cfProvider.getConfiguredProperties(), ImmutableMap.of("dynamicProperty", "dynamicValue", "hostName", HOSTNAME, "port", PORT));
     }
 }
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigIT.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigIT.java
new file mode 100644
index 0000000..f51288d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigIT.java
@@ -0,0 +1,176 @@
+/*
+ * 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.jms.processors;
+
+import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.nifi.jms.cf.JMSConnectionFactoryProperties;
+import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
+import org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties;
+import org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProvider;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.jms.Connection;
+import javax.jms.JMSException;
+
+/**
+ * Tests for the different Connection Factory configurations of {@link PublishJMS} and {@link ConsumeJMS}:
+ *     - JndiJmsConnectionFactoryProvider controller service
+ *     - JMSConnectionFactoryProvider controller service
+ *     - local JndiJmsConnectionFactory configuration on the processor
+ *     - local JMSConnectionFactory configuration on the processor
+ */
+public class ConnectionFactoryConfigIT {
+
+    private static final String CONTROLLER_SERVICE_ID = "cfProvider";
+
+    private static final String BROKER_URL = "vm://test-broker?broker.persistent=false";
+
+    private static final String PROP_JNDI_INITIAL_CONTEXT_FACTORY = "org.apache.activemq.jndi.ActiveMQInitialContextFactory";
+    private static final String PROP_JNDI_PROVIDER_URL = BROKER_URL;
+    private static final String PROP_JNDI_CONNECTION_FACTORY_NAME = "ConnectionFactory";
+
+    private static final String PROP_JMS_CONNECTION_FACTORY_IMPL = "org.apache.activemq.ActiveMQConnectionFactory";
+    private static final String PROP_JMS_BROKER_URI = BROKER_URL;
+
+    private static final String TEST_MESSAGE = "test-message";
+
+    private static Connection bootstrapConnection;
+
+    private TestRunner publisher;
+    private TestRunner consumer;
+
+    @BeforeClass
+    public static void beforeClass() throws JMSException {
+        // start in-VM broker
+        bootstrapConnection = new ActiveMQConnectionFactory(BROKER_URL).createConnection();
+    }
+
+    @AfterClass
+    public static void afterClass() throws JMSException {
+        // stop in-VM broker
+        bootstrapConnection.close();
+    }
+
+    @Before
+    public void before() {
+        publisher = TestRunners.newTestRunner(PublishJMS.class);
+        consumer = TestRunners.newTestRunner(ConsumeJMS.class);
+    }
+
+    @Test
+    public void testJndiJmsConnectionFactoryControllerService() throws InitializationException {
+        String queueName = "queue-jndi-service";
+
+        configureJndiJmsConnectionFactoryControllerService(publisher, queueName);
+        configureJndiJmsConnectionFactoryControllerService(consumer, queueName);
+
+        executeProcessors();
+
+        assertResult();
+    }
+
+    @Test
+    public void testJMSConnectionFactoryControllerService() throws InitializationException {
+        String queueName = "queue-jms-service";
+
+        configureJMSConnectionFactoryControllerService(publisher, queueName);
+        configureJMSConnectionFactoryControllerService(consumer, queueName);
+
+        executeProcessors();
+
+        assertResult();
+    }
+
+    @Test
+    public void testLocalJndiJmsConnectionFactoryConfig() {
+        String queueName = "queue-jndi-local";
+
+        configureLocalJndiJmsConnectionFactory(publisher, queueName);
+        configureLocalJndiJmsConnectionFactory(consumer, queueName);
+
+        executeProcessors();
+
+        assertResult();
+    }
+
+    @Test
+    public void testLocalJMSConnectionFactoryConfig() {
+        String queueName = "queue-jms-local";
+
+        configureLocalJMSConnectionFactory(publisher, queueName);
+        configureLocalJMSConnectionFactory(consumer, queueName);
+
+        executeProcessors();
+
+        assertResult();
+    }
+
+    private void configureJndiJmsConnectionFactoryControllerService(TestRunner runner, String queueName) throws InitializationException {
+        JndiJmsConnectionFactoryProvider cfProvider = new JndiJmsConnectionFactoryProvider();
+        runner.addControllerService(CONTROLLER_SERVICE_ID, cfProvider);
+        runner.setProperty(cfProvider, JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY, PROP_JNDI_INITIAL_CONTEXT_FACTORY);
+        runner.setProperty(cfProvider, JndiJmsConnectionFactoryProperties.JNDI_PROVIDER_URL, PROP_JNDI_PROVIDER_URL);
+        runner.setProperty(cfProvider, JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME, PROP_JNDI_CONNECTION_FACTORY_NAME);
+        runner.enableControllerService(cfProvider);
+        runner.setProperty(AbstractJMSProcessor.CF_SERVICE, CONTROLLER_SERVICE_ID);
+        runner.setProperty(AbstractJMSProcessor.DESTINATION, queueName);
+    }
+
+    private void configureJMSConnectionFactoryControllerService(TestRunner runner, String queueName) throws InitializationException {
+        JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
+        runner.addControllerService(CONTROLLER_SERVICE_ID, cfProvider);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, PROP_JMS_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, PROP_JMS_BROKER_URI);
+        runner.enableControllerService(cfProvider);
+        runner.setProperty(AbstractJMSProcessor.CF_SERVICE, CONTROLLER_SERVICE_ID);
+        runner.setProperty(AbstractJMSProcessor.DESTINATION, queueName);
+    }
+
+    private void configureLocalJndiJmsConnectionFactory(TestRunner runner, String queueName) {
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY, PROP_JNDI_INITIAL_CONTEXT_FACTORY);
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_PROVIDER_URL, PROP_JNDI_PROVIDER_URL);
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME, PROP_JNDI_CONNECTION_FACTORY_NAME);
+        runner.setProperty(AbstractJMSProcessor.DESTINATION, queueName);
+    }
+
+    private void configureLocalJMSConnectionFactory(TestRunner runner, String queueName) {
+        runner.setProperty(JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, PROP_JMS_CONNECTION_FACTORY_IMPL);
+        runner.setProperty(JMSConnectionFactoryProperties.JMS_BROKER_URI, PROP_JMS_BROKER_URI);
+        runner.setProperty(AbstractJMSProcessor.DESTINATION, queueName);
+    }
+
+    private void executeProcessors() {
+        publisher.enqueue(TEST_MESSAGE);
+        publisher.run();
+
+        consumer.run();
+    }
+
+    private void assertResult() {
+        publisher.assertAllFlowFilesTransferred(PublishJMS.REL_SUCCESS, 1);
+        publisher.getFlowFilesForRelationship(ConsumeJMS.REL_SUCCESS).get(0).assertContentEquals(TEST_MESSAGE);
+
+        consumer.assertAllFlowFilesTransferred(ConsumeJMS.REL_SUCCESS, 1);
+        consumer.getFlowFilesForRelationship(ConsumeJMS.REL_SUCCESS).get(0).assertContentEquals(TEST_MESSAGE);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigValidatorTest.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigValidatorTest.java
new file mode 100644
index 0000000..339a1f1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConnectionFactoryConfigValidatorTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.jms.processors;
+
+import org.apache.nifi.jms.cf.JMSConnectionFactoryProperties;
+import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
+import org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests for {@link AbstractJMSProcessor.ConnectionFactoryConfigValidator}
+ */
+public class ConnectionFactoryConfigValidatorTest {
+
+    private static final String CONTROLLER_SERVICE_ID = "cfProvider";
+
+    private static final String PROP_JNDI_INITIAL_CONTEXT_FACTORY = "org.apache.activemq.jndi.ActiveMQInitialContextFactory";
+    private static final String PROP_JNDI_PROVIDER_URL = "tcp://myhost:61616";
+    private static final String PROP_JNDI_CONNECTION_FACTORY_NAME = "ConnectionFactory";
+
+    private static final String PROP_JMS_CONNECTION_FACTORY_IMPL = "org.apache.activemq.ActiveMQConnectionFactory";
+    private static final String PROP_JMS_BROKER_URI = "tcp://myhost:61616";
+
+    private TestRunner runner;
+
+    @Before
+    public void setUp() {
+        runner = TestRunners.newTestRunner(PublishJMS.class);
+        runner.setProperty(PublishJMS.DESTINATION, "myQueue");
+    }
+
+    @Test
+    public void testNotValidWhenNoConnectionFactoryConfigured() {
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testValidControllerServiceConfig() throws InitializationException {
+        configureControllerService();
+
+        runner.assertValid();
+    }
+
+    @Test
+    public void testNotValidWhenControllerServiceConfiguredButLocalJndiJmsConnectionFactoryPropertyAlsoSpecified() throws InitializationException {
+        configureControllerService();
+
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME, PROP_JNDI_CONNECTION_FACTORY_NAME);
+
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testNotValidWhenControllerServiceConfiguredButLocalJMSConnectionFactoryPropertyAlsoSpecified() throws InitializationException {
+        configureControllerService();
+
+        runner.setProperty(JMSConnectionFactoryProperties.JMS_BROKER_URI, PROP_JMS_BROKER_URI);
+
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testValidLocalJndiJmsConnectionFactoryConfig() {
+        configureLocalJndiJmsConnectionFactory();
+
+        runner.assertValid();
+    }
+
+    @Test
+    public void testNotValidWhenLocalJndiJmsConnectionFactoryConfiguredButLocalJMSConnectionFactoryPropertyAlsoSpecified() {
+        configureLocalJndiJmsConnectionFactory();
+
+        runner.setProperty(JMSConnectionFactoryProperties.JMS_BROKER_URI, PROP_JMS_BROKER_URI);
+
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testNotValidWhenNoProviderUrlSpecifiedForLocalJndiJmsConnectionFactory() {
+        configureLocalJndiJmsConnectionFactory();
+        runner.removeProperty(JndiJmsConnectionFactoryProperties.JNDI_PROVIDER_URL);
+
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testNotValidWhenNoConnectionFactoryNameSpecifiedForLocalJndiJmsConnectionFactory() {
+        configureLocalJndiJmsConnectionFactory();
+        runner.removeProperty(JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME);
+
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testValidLocalJMSConnectionFactory() {
+        configureLocalJMSConnectionFactory();
+
+        runner.assertValid();
+    }
+
+    @Test
+    public void testNotValidWhenLocalJMSConnectionFactoryConfiguredButLocalJndiJmsConnectionFactoryPropertyAlsoSpecified() {
+        configureLocalJMSConnectionFactory();
+
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME, PROP_JNDI_CONNECTION_FACTORY_NAME);
+
+        runner.assertNotValid();
+    }
+
+    private void configureControllerService() throws InitializationException {
+        JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
+        runner.addControllerService(CONTROLLER_SERVICE_ID, cfProvider);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, PROP_JMS_CONNECTION_FACTORY_IMPL);
+        runner.enableControllerService(cfProvider);
+        runner.setProperty(AbstractJMSProcessor.CF_SERVICE, CONTROLLER_SERVICE_ID);
+    }
+
+    private void configureLocalJndiJmsConnectionFactory() {
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY, PROP_JNDI_INITIAL_CONTEXT_FACTORY);
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_PROVIDER_URL, PROP_JNDI_PROVIDER_URL);
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME, PROP_JNDI_CONNECTION_FACTORY_NAME);
+    }
+
+    private void configureLocalJMSConnectionFactory() {
+        runner.setProperty(JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, PROP_JMS_CONNECTION_FACTORY_IMPL);
+    }
+}


[nifi] 29/47: NIFI-7245: JMS processors yield when connection factory initialisation failed

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 833eea4547fa076723a1ff9d7bdcae5f37bbf497
Author: Peter Turcsanyi <tu...@apache.org>
AuthorDate: Wed Mar 11 15:46:05 2020 +0100

    NIFI-7245: JMS processors yield when connection factory initialisation failed
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4133.
---
 .../nifi/jms/processors/AbstractJMSProcessor.java  |  8 ++++++-
 .../apache/nifi/jms/processors/ConsumeJMSIT.java   | 26 ++++++++++++++++++++++
 .../apache/nifi/jms/processors/PublishJMSIT.java   | 25 +++++++++++++++++++++
 3 files changed, 58 insertions(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index 39b8dac..78b2d90 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -179,7 +179,13 @@ public abstract class AbstractJMSProcessor<T extends JMSWorker> extends Abstract
     public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
         T worker = workerPool.poll();
         if (worker == null) {
-            worker = buildTargetResource(context);
+            try {
+                worker = buildTargetResource(context);
+            } catch (Exception e) {
+                getLogger().error("Failed to initialize JMS Connection Factory", e);
+                context.yield();
+                return;
+            }
         }
 
         try {
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java
index 714b950..64728fe 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSIT.java
@@ -31,6 +31,8 @@ import org.apache.activemq.command.ActiveMQMessage;
 import org.apache.activemq.transport.tcp.TcpTransport;
 import org.apache.activemq.transport.tcp.TcpTransportFactory;
 import org.apache.activemq.wireformat.WireFormat;
+import org.apache.nifi.jms.cf.JMSConnectionFactoryProperties;
+import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProviderDefinition;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
@@ -418,6 +420,30 @@ public class ConsumeJMSIT {
         }
     }
 
+    @Test
+    public void whenExceptionIsRaisedDuringConnectionFactoryInitializationTheProcessorShouldBeYielded() throws Exception {
+        TestRunner runner = TestRunners.newTestRunner(ConsumeJMS.class);
+
+        // using (non-JNDI) JMS Connection Factory via controller service
+        JMSConnectionFactoryProvider cfProvider = new JMSConnectionFactoryProvider();
+        runner.addControllerService("cfProvider", cfProvider);
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL, "DummyJMSConnectionFactoryClass");
+        runner.setProperty(cfProvider, JMSConnectionFactoryProperties.JMS_BROKER_URI, "DummyBrokerUri");
+        runner.enableControllerService(cfProvider);
+
+        runner.setProperty(ConsumeJMS.CF_SERVICE, "cfProvider");
+        runner.setProperty(ConsumeJMS.DESTINATION, "myTopic");
+        runner.setProperty(ConsumeJMS.DESTINATION_TYPE, ConsumeJMS.TOPIC);
+
+        try {
+            runner.run();
+            fail("The test was implemented in a way this line should not be reached.");
+        } catch (AssertionError e) {
+        } finally {
+            assertTrue("In case of an exception, the processor should be yielded.", ((MockProcessContext) runner.getProcessContext()).isYieldCalled());
+        }
+    }
+
     private static void publishAMessage(ActiveMQConnectionFactory cf, final String destinationName, String messageContent) throws JMSException {
         // Publish a message.
         try (Connection conn = cf.createConnection();
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java
index 247a3ac..b901b73 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -31,10 +32,12 @@ import org.apache.activemq.transport.tcp.TcpTransport;
 import org.apache.activemq.transport.tcp.TcpTransportFactory;
 import org.apache.activemq.wireformat.WireFormat;
 import org.apache.nifi.jms.cf.JMSConnectionFactoryProviderDefinition;
+import org.apache.nifi.jms.cf.JndiJmsConnectionFactoryProperties;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.MockProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
@@ -381,4 +384,26 @@ public class PublishJMSIT {
             }
         }
     }
+
+    @Test
+    public void whenExceptionIsRaisedDuringConnectionFactoryInitializationTheProcessorShouldBeYielded() throws Exception {
+        TestRunner runner = TestRunners.newTestRunner(PublishJMS.class);
+
+        // using JNDI JMS Connection Factory configured locally on the processor
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_INITIAL_CONTEXT_FACTORY, "DummyInitialContextFactoryClass");
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_PROVIDER_URL, "DummyProviderUrl");
+        runner.setProperty(JndiJmsConnectionFactoryProperties.JNDI_CONNECTION_FACTORY_NAME, "DummyConnectionFactoryName");
+
+        runner.setProperty(ConsumeJMS.DESTINATION, "myTopic");
+        runner.setProperty(ConsumeJMS.DESTINATION_TYPE, ConsumeJMS.TOPIC);
+
+        try {
+            runner.enqueue("message");
+            runner.run();
+            fail("The test was implemented in a way this line should not be reached.");
+        } catch (AssertionError e) {
+        } finally {
+            assertTrue("In case of an exception, the processor should be yielded.", ((MockProcessContext) runner.getProcessContext()).isYieldCalled());
+        }
+    }
 }


[nifi] 44/47: NIFI-7223 - Fixed a minor issue where the OkHttpReplicationClient class loaded blank properties as empty string instead of an expected null value. Added a isNotBlank check. Added unit tests for replication client and HTTPNotificationService. NIFI-7223 - Renamed some variables and methods. NIFI-7223 - Removed unused dependency. Corrected security properties in administration-guide.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 67db74148e2f73e27ce4a63308380bc7a0d50022
Author: Nathan Gough <th...@gmail.com>
AuthorDate: Tue Mar 10 21:22:29 2020 +1100

    NIFI-7223 - Fixed a minor issue where the OkHttpReplicationClient class loaded blank properties as empty string instead of an expected null value. Added a isNotBlank check. Added unit tests for replication client and HTTPNotificationService.
    NIFI-7223 - Renamed some variables and methods.
    NIFI-7223 - Removed unused dependency. Corrected security properties in administration-guide.
---
 nifi-bootstrap/pom.xml                             |   5 +
 .../http/TestHttpNotificationServiceSSL.java       | 190 ++++++++++++++++++++-
 .../src/main/asciidoc/administration-guide.adoc    |   5 +-
 .../okhttp/OkHttpReplicationClient.java            |  11 +-
 .../okhttp/OkHttpReplicationClientTest.groovy      |  75 ++++++++
 .../src/test/resources/conf/keystore.jks           | Bin 0 -> 3088 bytes
 .../src/test/resources/conf/truststore.jks         | Bin 0 -> 911 bytes
 7 files changed, 273 insertions(+), 13 deletions(-)

diff --git a/nifi-bootstrap/pom.xml b/nifi-bootstrap/pom.xml
index 5d8dbdf..a1a9134 100644
--- a/nifi-bootstrap/pom.xml
+++ b/nifi-bootstrap/pom.xml
@@ -73,5 +73,10 @@ language governing permissions and limitations under the License. -->
             <artifactId>jna-platform</artifactId>
             <version>4.4.0</version>
         </dependency>
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
index 5ab8cab..2f54f1c 100644
--- a/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
+++ b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
@@ -16,13 +16,21 @@
  */
 package org.apache.nifi.bootstrap.http;
 
+import ch.qos.logback.classic.spi.ILoggingEvent;
+import ch.qos.logback.core.read.ListAppender;
 import okhttp3.mockwebserver.MockWebServer;
+import org.apache.nifi.bootstrap.NotificationServiceManager;
 import org.apache.nifi.security.util.SslContextFactory;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 import org.mockito.internal.util.io.IOUtil;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
+import ch.qos.logback.classic.Logger;
 
 import javax.net.ssl.SSLContext;
+import javax.xml.parsers.ParserConfigurationException;
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
@@ -32,9 +40,12 @@ import java.security.KeyStoreException;
 import java.security.NoSuchAlgorithmException;
 import java.security.UnrecoverableKeyException;
 import java.security.cert.CertificateException;
+import java.util.List;
 
-public class TestHttpNotificationServiceSSL extends  TestHttpNotificationServiceCommon{
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
+public class TestHttpNotificationServiceSSL extends TestHttpNotificationServiceCommon {
 
     static final String CONFIGURATION_FILE_TEXT = "\n"+
             "<services>\n"+
@@ -47,14 +58,80 @@ public class TestHttpNotificationServiceSSL extends  TestHttpNotificationService
             "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
             "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
             "            <property name=\"Keystore Type\">JKS</property>\n"+
+            "            <property name=\"Key Password\">passwordpassword</property>\n"+
             "            <property name=\"Keystore Password\">passwordpassword</property>\n"+
             "            <property name=\"testProp\">${literal('testing')}</property>\n"+
             "         </service>\n"+
             "</services>";
 
+    static final String CONFIGURATION_FILE_TEXT_NO_KEYSTORE_PASSWORD = "\n"+
+            "<services>\n"+
+            "         <service>\n"+
+            "            <id>http-notification</id>\n"+
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
+            "            <property name=\"URL\">${test.server}</property>\n"+
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
+            "            <property name=\"Truststore Type\">JKS</property>\n"+
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
+            "            <property name=\"Keystore Type\">JKS</property>\n"+
+            "            <property name=\"Key Password\">passwordpassword</property>\n"+
+            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
+            "         </service>\n"+
+            "</services>";
+
+    static final String CONFIGURATION_FILE_TEXT_NO_KEY_PASSWORD = "\n"+
+            "<services>\n"+
+            "         <service>\n"+
+            "            <id>http-notification</id>\n"+
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
+            "            <property name=\"URL\">${test.server}</property>\n"+
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
+            "            <property name=\"Truststore Type\">JKS</property>\n"+
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
+            "            <property name=\"Keystore Type\">JKS</property>\n"+
+            "            <property name=\"Keystore Password\">passwordpassword</property>\n"+
+            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
+            "         </service>\n"+
+            "</services>";
+
+    static final String CONFIGURATION_FILE_TEXT_BLANK_KEY_PASSWORD = "\n"+
+            "<services>\n"+
+            "         <service>\n"+
+            "            <id>http-notification</id>\n"+
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
+            "            <property name=\"URL\">${test.server}</property>\n"+
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
+            "            <property name=\"Truststore Type\">JKS</property>\n"+
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
+            "            <property name=\"Keystore Type\">JKS</property>\n"+
+            "            <property name=\"Keystore Password\">passwordpassword</property>\n"+
+            "            <property name=\"Key Password\"></property>\n"+
+            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
+            "         </service>\n"+
+            "</services>";
 
-    @BeforeClass
-    public static void startServer() throws IOException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException {
+    static final String CONFIGURATION_FILE_TEXT_BLANK_KEYSTORE_PASSWORD = "\n"+
+            "<services>\n"+
+            "         <service>\n"+
+            "            <id>http-notification</id>\n"+
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
+            "            <property name=\"URL\">${test.server}</property>\n"+
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
+            "            <property name=\"Truststore Type\">JKS</property>\n"+
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
+            "            <property name=\"Keystore Type\">JKS</property>\n"+
+            "            <property name=\"Keystore Password\"></property>\n"+
+            "            <property name=\"Key Password\">passwordpassword</property>\n"+
+            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
+            "         </service>\n"+
+            "</services>";
+
+    @Before
+    public void startServer() throws IOException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException {
         tempConfigFilePath = "./target/TestHttpNotificationService-config.xml";
 
         Files.deleteIfExists(Paths.get(tempConfigFilePath));
@@ -78,10 +155,109 @@ public class TestHttpNotificationServiceSSL extends  TestHttpNotificationService
         IOUtil.writeText(configFileOutput, new File(tempConfigFilePath));
     }
 
-    @AfterClass
-    public static void shutdownServer() throws IOException {
+    @After
+    public void shutdownServer() throws IOException {
         Files.deleteIfExists(Paths.get(tempConfigFilePath));
         mockWebServer.shutdown();
     }
 
+    @Test
+    public void testStartNotificationSucceedsNoKeystorePasswd() throws ParserConfigurationException, SAXException, IOException {
+        Logger notificationServiceLogger = (Logger) LoggerFactory.getLogger(NotificationServiceManager.class);
+        ListAppender<ILoggingEvent> listAppender = new ListAppender<>();
+        listAppender.start();
+        notificationServiceLogger.addAppender(listAppender);
+
+        String configFileOutput = CONFIGURATION_FILE_TEXT_NO_KEYSTORE_PASSWORD.replace("${test.server}", String.valueOf(mockWebServer.url("/")));
+        IOUtil.writeText(configFileOutput, new File(tempConfigFilePath));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+
+        List<ILoggingEvent> logsList = listAppender.list;
+        boolean notificationServiceFailed = false;
+        for(ILoggingEvent logMessage : logsList) {
+            if(logMessage.getFormattedMessage().contains("is not valid for the following reasons")) {
+                    notificationServiceFailed = true;
+            }
+        }
+
+        assertFalse(notificationServiceFailed);
+    }
+
+    @Test
+    public void testStartNotificationSucceedsNoKeyPasswd() throws ParserConfigurationException, SAXException, IOException {
+        Logger notificationServiceLogger = (Logger) LoggerFactory.getLogger(NotificationServiceManager.class);
+        ListAppender<ILoggingEvent> listAppender = new ListAppender<>();
+        listAppender.start();
+        notificationServiceLogger.addAppender(listAppender);
+
+        String configFileOutput = CONFIGURATION_FILE_TEXT_NO_KEY_PASSWORD.replace("${test.server}", String.valueOf(mockWebServer.url("/")));
+        IOUtil.writeText(configFileOutput, new File(tempConfigFilePath));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+
+        List<ILoggingEvent> logsList = listAppender.list;
+        boolean notificationServiceFailed = false;
+        for(ILoggingEvent logMessage : logsList) {
+            if(logMessage.getFormattedMessage().contains("is not valid for the following reasons")) {
+                notificationServiceFailed = true;
+            }
+        }
+
+        assertFalse(notificationServiceFailed);
+    }
+
+    @Test
+    public void testStartNotificationFailsBlankKeystorePasswdCorrectKeypasswd() throws ParserConfigurationException, SAXException, IOException {
+        Logger notificationServiceLogger = (Logger) LoggerFactory.getLogger(NotificationServiceManager.class);
+        ListAppender<ILoggingEvent> listAppender = new ListAppender<>();
+        listAppender.start();
+        notificationServiceLogger.addAppender(listAppender);
+
+        String configFileOutput = CONFIGURATION_FILE_TEXT_BLANK_KEYSTORE_PASSWORD.replace("${test.server}", String.valueOf(mockWebServer.url("/")));
+        IOUtil.writeText(configFileOutput, new File(tempConfigFilePath));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+
+        List<ILoggingEvent> logsList = listAppender.list;
+        boolean notificationServiceFailed = false;
+        for(ILoggingEvent logMessage : logsList) {
+            if(logMessage.getFormattedMessage().contains("'Keystore Password' validated against '' is invalid because Keystore Password cannot be empty")) {
+                notificationServiceFailed = true;
+            }
+        }
+
+        assertTrue(notificationServiceFailed);
+    }
+
+    @Test
+    public void testStartNotificationFailsCorrectKeystorePasswdBlankKeypasswd() throws ParserConfigurationException, SAXException, IOException {
+        Logger notificationServiceLogger = (Logger) LoggerFactory.getLogger(NotificationServiceManager.class);
+        ListAppender<ILoggingEvent> listAppender = new ListAppender<>();
+        listAppender.start();
+        notificationServiceLogger.addAppender(listAppender);
+
+        String configFileOutput = CONFIGURATION_FILE_TEXT_BLANK_KEY_PASSWORD.replace("${test.server}", String.valueOf(mockWebServer.url("/")));
+        IOUtil.writeText(configFileOutput, new File(tempConfigFilePath));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+
+        List<ILoggingEvent> logsList = listAppender.list;
+        boolean notificationServiceFailed = false;
+        for(ILoggingEvent logMessage : logsList) {
+            if(logMessage.getFormattedMessage().contains("'Key Password' validated against '' is invalid because Key Password cannot be empty")) {
+                notificationServiceFailed = true;
+            }
+        }
+
+        assertTrue(notificationServiceFailed);
+    }
 }
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 5c90725..c97aa89 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -2252,8 +2252,9 @@ It has the following properties available:
 |`Truststore Type`||The Type of the Truststore. Either `JKS` or `PKCS12`
 |`Truststore Password`||The password for the Truststore
 |`Keystore Filename`||The fully-qualified filename of the Keystore
-|`Keystore Type`||The password for the Keystore
-|`Keystore Password`||The password for the key. If this is not specified, but the Keystore Filename, Password, and Type are specified, then the Keystore Password will be assumed to be the same as the Key Password.
+|`Keystore Type`||The Type of the Keystore. Either `JKS` or `PKCS12`
+|`Keystore Password`||The password for the Keystore
+|`Key Password`||The password for the key. If this is not specified, but the Keystore Filename, Password, and Type are specified, then the Key Password will be assumed to be the same as the Keystore Password.
 |`SSL Protocol`||The algorithm to use for this SSL context. This can either be `SSL` or `TLS`.
 |====
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
index 2352026..ff63836 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
@@ -334,11 +334,11 @@ public class OkHttpReplicationClient implements HttpReplicationClient {
         try {
             Tuple<SSLContext, TrustManager[]> sslContextTuple = createTrustSslContextWithTrustManagers(
                     properties.getProperty(NiFiProperties.SECURITY_KEYSTORE),
-                    properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD) != null ? properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD).toCharArray() : null,
-                    properties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD) != null ? properties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD).toCharArray() : null,
+                    StringUtils.isNotBlank(properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD)) ? properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD).toCharArray() : null,
+                    StringUtils.isNotBlank(properties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD)) ? properties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD).toCharArray() : null,
                     properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE),
                     properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE),
-                    properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD) != null ? properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD).toCharArray() : null,
+                    StringUtils.isNotBlank(properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD)) ? properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD).toCharArray() : null,
                     properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE),
                     WANT,
                     sslContext.getProtocol());
@@ -346,7 +346,10 @@ public class OkHttpReplicationClient implements HttpReplicationClient {
                     .filter(trustManager -> trustManager instanceof X509TrustManager)
                     .map(trustManager -> (X509TrustManager) trustManager).collect(Collectors.toList());
             return new Tuple<>(sslContextTuple.getKey().getSocketFactory(), x509TrustManagers.get(0));
-        } catch (CertificateException | UnrecoverableKeyException | NoSuchAlgorithmException | KeyStoreException | KeyManagementException | IOException e) {
+        } catch(UnrecoverableKeyException e) {
+            logger.error("Key password may be incorrect or not set. Check your keystore passwords." + e.getMessage());
+            return null;
+        } catch (CertificateException | NoSuchAlgorithmException | KeyStoreException | KeyManagementException | IOException e) {
             return null;
         }
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
index cad27f1..712f80a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
@@ -26,6 +26,8 @@ import org.junit.runner.RunWith
 import org.junit.runners.JUnit4
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
+import sun.security.ssl.DummyX509KeyManager
+import sun.security.ssl.SunX509KeyManagerImpl
 
 @RunWith(JUnit4.class)
 class OkHttpReplicationClientTest extends GroovyTestCase {
@@ -135,4 +137,77 @@ class OkHttpReplicationClientTest extends GroovyTestCase {
             assert headers."Content-Length" == "123"
         }
     }
+
+    @Test
+    void testShouldUseKeystorePasswdIfKeypasswdIsBlank() {
+        // Arrange
+        Map flowfileEncryptionProps = [
+                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEY_PASSWD): "",
+                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+        ]
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
+
+        // Act
+        OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+
+        // Assert
+        assertNotNull(client.okHttpClient.sslSocketFactory)
+        assertEquals(SunX509KeyManagerImpl.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
+        assertNotNull(client.okHttpClient.sslSocketFactory.context.getX509KeyManager().credentialsMap["nifi-key"])
+    }
+
+    @Test
+    void testShouldFailIfKeyPasswdIsSetButKeystorePasswdIsBlank() {
+        // Arrange
+        Map flowfileEncryptionProps = [
+                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "",
+                (NiFiProperties.SECURITY_KEY_PASSWD): "passwordpassword",
+                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+        ]
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
+
+        // Act
+        OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+
+        // Assert
+        // The replication client will fail to initialize if the keystore password is missing, and will use
+        // a default empty DummyX509KeyManager instead. This is considered a failure to start the service.
+        assertSame(DummyX509KeyManager.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
+    }
+
+    @Test
+    void testShouldFailIfKeyPasswdIsBlankAndKeystorePasswd() {
+        // Arrange
+        Map flowfileEncryptionProps = [
+                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "",
+                (NiFiProperties.SECURITY_KEY_PASSWD): "",
+                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+        ]
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
+
+        // Act
+        OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+
+        // Assert
+        assertEquals(DummyX509KeyManager.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/keystore.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/keystore.jks
new file mode 100644
index 0000000..246fe88
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/keystore.jks differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/truststore.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/truststore.jks
new file mode 100644
index 0000000..87f4be1
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/truststore.jks differ


[nifi] 14/47: NIFI-7222 Cleaned up API for FTP/SFTP remote file retrieval and ensure we close remote file resources for SFTP pulls in particular

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 4c78d1e92b77568108c8fe8da9c086040ad41213
Author: Joe Witt <jo...@apache.org>
AuthorDate: Wed Mar 4 23:41:45 2020 -0500

    NIFI-7222 Cleaned up API for FTP/SFTP remote file retrieval and ensure we close remote file resources for SFTP pulls in particular
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4115.
---
 .../processors/standard/FetchFileTransfer.java     | 18 +------
 .../nifi/processors/standard/GetFileTransfer.java  | 10 ++--
 .../nifi/processors/standard/util/FTPTransfer.java | 62 ++++++++++++----------
 .../processors/standard/util/FileTransfer.java     | 10 ++--
 .../processors/standard/util/SFTPTransfer.java     | 57 ++++++++++++--------
 .../processors/standard/TestFetchFileTransfer.java | 31 +++++------
 .../util/ITestSFTPTransferWithSSHTestServer.java   | 23 --------
 7 files changed, 91 insertions(+), 120 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
index b7e8f61..b975548 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
@@ -31,19 +31,15 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.standard.util.FileTransfer;
 import org.apache.nifi.processors.standard.util.PermissionDeniedException;
-import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.StopWatch;
 import org.apache.nifi.util.Tuple;
 
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -261,20 +257,8 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         boolean closeConnection = false;
         try {
             // Pull data from remote system.
-            final InputStream in;
             try {
-                in = transfer.getInputStream(filename, flowFile);
-
-                flowFile = session.write(flowFile, new OutputStreamCallback() {
-                    @Override
-                    public void process(final OutputStream out) throws IOException {
-                        StreamUtils.copy(in, out);
-                    }
-                });
-
-                if (!transfer.flush(flowFile)) {
-                    throw new IOException("completePendingCommand returned false, file transfer failed");
-                }
+                flowFile = transfer.getRemoteFile(filename, flowFile, session);
 
             } catch (final FileNotFoundException e) {
                 closeConnection = false;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java
index 4ce31de..8ac5a27 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java
@@ -18,7 +18,6 @@ package org.apache.nifi.processors.standard;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.nio.file.Path;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -189,12 +188,9 @@ public abstract class GetFileTransfer extends AbstractProcessor {
                 try {
                     FlowFile flowFile = session.create();
                     final StopWatch stopWatch = new StopWatch(false);
-                    try (final InputStream in = transfer.getInputStream(file.getFullPathFileName())) {
-                        stopWatch.start();
-                        flowFile = session.importFrom(in, flowFile);
-                        stopWatch.stop();
-                    }
-                    transfer.flush();
+                    stopWatch.start();
+                    flowFile = transfer.getRemoteFile(file.getFullPathFileName(), flowFile, session);
+                    stopWatch.stop();
                     final long millis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
                     final String dataRate = stopWatch.calculateDataRate(flowFile.getSize());
                     flowFile = session.putAttribute(flowFile, this.getClass().getSimpleName().toLowerCase() + ".remote.source", hostname);
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
index 6a2b2db..e7d96c6 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
@@ -20,6 +20,7 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Proxy;
@@ -50,10 +51,13 @@ import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.proxy.ProxyConfiguration;
 import org.apache.nifi.proxy.ProxySpec;
+import org.apache.nifi.stream.io.StreamUtils;
 
 public class FTPTransfer implements FileTransfer {
 
@@ -314,35 +318,39 @@ public class FTPTransfer implements FileTransfer {
     }
 
     @Override
-    public InputStream getInputStream(String remoteFileName) throws IOException {
-        return getInputStream(remoteFileName, null);
-    }
-
-    @Override
-    public InputStream getInputStream(final String remoteFileName, final FlowFile flowFile) throws IOException {
-        final FTPClient client = getClient(flowFile);
-        InputStream in = client.retrieveFileStream(remoteFileName);
-        if (in == null) {
-            final String response = client.getReplyString();
-            // FTPClient doesn't throw exception if file not found.
-            // Instead, response string will contain: "550 Can't open <absolute_path>: No such file or directory"
-            if (response != null && response.trim().endsWith("No such file or directory")){
-                throw new FileNotFoundException(response);
+    public FlowFile getRemoteFile(final String remoteFileName, final FlowFile origFlowFile, final ProcessSession session) throws ProcessException, IOException {
+        final FTPClient client = getClient(origFlowFile);
+        InputStream in = null;
+        FlowFile resultFlowFile = null;
+        try {
+            in = client.retrieveFileStream(remoteFileName);
+            if (in == null) {
+                final String response = client.getReplyString();
+                // FTPClient doesn't throw exception if file not found.
+                // Instead, response string will contain: "550 Can't open <absolute_path>: No such file or directory"
+                if (response != null && response.trim().endsWith("No such file or directory")) {
+                    throw new FileNotFoundException(response);
+                }
+                throw new IOException(response);
+            }
+            final InputStream remoteIn = in;
+            resultFlowFile = session.write(origFlowFile, new OutputStreamCallback() {
+                @Override
+                public void process(final OutputStream out) throws IOException {
+                    StreamUtils.copy(remoteIn, out);
+                }
+            });
+            client.completePendingCommand();
+            return resultFlowFile;
+        } finally {
+            if(in != null){
+                try{
+                    in.close();
+                }catch(final IOException ioe){
+                    //do nothing
+                }
             }
-            throw new IOException(response);
         }
-        return in;
-    }
-
-    @Override
-    public void flush() throws IOException {
-        final FTPClient client = getClient(null);
-        client.completePendingCommand();
-    }
-
-    @Override
-    public boolean flush(final FlowFile flowFile) throws IOException {
-        return getClient(flowFile).completePendingCommand();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
index 56dd22d..64bb130 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
@@ -26,6 +26,8 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 
 public interface FileTransfer extends Closeable {
@@ -34,13 +36,7 @@ public interface FileTransfer extends Closeable {
 
     List<FileInfo> getListing() throws IOException;
 
-    InputStream getInputStream(String remoteFileName) throws IOException;
-
-    InputStream getInputStream(String remoteFileName, FlowFile flowFile) throws IOException;
-
-    void flush() throws IOException;
-
-    boolean flush(FlowFile flowFile) throws IOException;
+    FlowFile getRemoteFile(String remoteFileName, FlowFile flowFile, ProcessSession session) throws ProcessException, IOException;
 
     FileInfo getRemoteFileInfo(FlowFile flowFile, String path, String remoteFileName) throws IOException;
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
index 465bdde..3a341bc 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
@@ -45,18 +45,23 @@ import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.proxy.ProxyConfiguration;
 import org.apache.nifi.proxy.ProxySpec;
+import org.apache.nifi.stream.io.StreamUtils;
 
 import javax.net.SocketFactory;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.Proxy;
 import java.net.Socket;
@@ -346,19 +351,22 @@ public class SFTPTransfer implements FileTransfer {
     }
 
     @Override
-    public InputStream getInputStream(final String remoteFileName) throws IOException {
-        return getInputStream(remoteFileName, null);
-    }
-
-    @Override
-    public InputStream getInputStream(final String remoteFileName, final FlowFile flowFile) throws IOException {
-        final SFTPClient sftpClient = getSFTPClient(flowFile);
+    public FlowFile getRemoteFile(final String remoteFileName, final FlowFile origFlowFile, final ProcessSession session) throws ProcessException, IOException {
+        final SFTPClient sftpClient = getSFTPClient(origFlowFile);
+        RemoteFile rf = null;
+        RemoteFile.ReadAheadRemoteFileInputStream rfis = null;
+        FlowFile resultFlowFile = null;
         try {
-            // The client has 'get' methods for downloading a file, but they don't offer a way to get access to an InputStream so
-            // this code is what the SFTPTransfer Downloader does to get a stream for the remote file contents
-            final RemoteFile rf = sftpClient.open(remoteFileName);
-            final RemoteFile.ReadAheadRemoteFileInputStream rfis = rf.new ReadAheadRemoteFileInputStream(16);
-            return rfis;
+            rf = sftpClient.open(remoteFileName);
+            rfis = rf.new ReadAheadRemoteFileInputStream(16);
+            final InputStream in = rfis;
+            resultFlowFile = session.write(origFlowFile, new OutputStreamCallback() {
+                @Override
+                public void process(final OutputStream out) throws IOException {
+                    StreamUtils.copy(in, out);
+                }
+            });
+            return resultFlowFile;
         } catch (final SFTPException e) {
             switch (e.getStatusCode()) {
                 case NO_SUCH_FILE:
@@ -368,20 +376,25 @@ public class SFTPTransfer implements FileTransfer {
                 default:
                     throw new IOException("Failed to obtain file content for " + remoteFileName, e);
             }
+        } finally {
+            if(rf != null){
+                try{
+                    rf.close();
+                }catch(final IOException ioe){
+                    //do nothing
+                }
+            }
+            if(rfis != null){
+                try{
+                    rfis.close();
+                }catch(final IOException ioe){
+                    //do nothing
+                }
+            }
         }
     }
 
     @Override
-    public void flush() throws IOException {
-        // nothing needed here
-    }
-
-    @Override
-    public boolean flush(final FlowFile flowFile) throws IOException {
-        return true;
-    }
-
-    @Override
     public void deleteFile(final FlowFile flowFile, final String path, final String remoteFileName) throws IOException {
         final SFTPClient sftpClient = getSFTPClient(flowFile);
         final String fullPath = (path == null) ? remoteFileName : (path.endsWith("/")) ? path + remoteFileName : path + "/" + remoteFileName;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
index de76b07..68eb627 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
@@ -26,6 +26,7 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -33,9 +34,13 @@ import java.util.Map;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processors.standard.util.FileInfo;
 import org.apache.nifi.processors.standard.util.FileTransfer;
 import org.apache.nifi.processors.standard.util.PermissionDeniedException;
+import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -285,12 +290,7 @@ public class TestFetchFileTransfer {
                 }
 
                 @Override
-                public InputStream getInputStream(final String remoteFileName) throws IOException {
-                    return getInputStream(remoteFileName, null);
-                }
-
-                @Override
-                public InputStream getInputStream(String remoteFileName, FlowFile flowFile) throws IOException {
+                public FlowFile getRemoteFile(String remoteFileName, FlowFile flowFile, ProcessSession session) throws ProcessException, IOException {
                     if (!allowAccess) {
                         throw new PermissionDeniedException("test permission denied");
                     }
@@ -299,17 +299,14 @@ public class TestFetchFileTransfer {
                     if (content == null) {
                         throw new FileNotFoundException();
                     }
-
-                    return new ByteArrayInputStream(content);
-                }
-
-                @Override
-                public void flush() throws IOException {
-                }
-
-                @Override
-                public boolean flush(FlowFile flowFile) throws IOException {
-                    return true;
+                    final InputStream in = new ByteArrayInputStream(content);
+                    flowFile = session.write(flowFile, new OutputStreamCallback() {
+                        @Override
+                        public void process(final OutputStream out) throws IOException {
+                            StreamUtils.copy(in, out);
+                        }
+                    });
+                    return flowFile;
                 }
 
                 @Override
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/ITestSFTPTransferWithSSHTestServer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/ITestSFTPTransferWithSSHTestServer.java
index 17a3cda..de8c8b6 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/ITestSFTPTransferWithSSHTestServer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/ITestSFTPTransferWithSSHTestServer.java
@@ -311,29 +311,6 @@ public class ITestSFTPTransferWithSSHTestServer {
     }
 
     @Test
-    public void testGetInputStream() throws IOException {
-        final String filename = "./" + DIR_2 + "/" + FILE_1;
-        final Map<PropertyDescriptor, String> properties = createBaseProperties();
-
-        try(final SFTPTransfer transfer = createSFTPTransfer(properties);
-            final InputStream in = transfer.getInputStream(filename)) {
-            final String content = IOUtils.toString(in, StandardCharsets.UTF_8);
-            assertEquals("dir2 file1", content);
-        }
-    }
-
-    @Test(expected = FileNotFoundException.class)
-    public void testGetInputStreamWhenFileDoesNotExist() throws IOException {
-        final String filename = "./" + DIR_2 + "/DOES-NOT-EXIST";
-        final Map<PropertyDescriptor, String> properties = createBaseProperties();
-
-        try(final SFTPTransfer transfer = createSFTPTransfer(properties);
-            final InputStream in = transfer.getInputStream(filename)) {
-            IOUtils.toString(in, StandardCharsets.UTF_8);
-        }
-    }
-
-    @Test
     public void testDeleteFileWithoutPath() throws IOException {
         final Map<PropertyDescriptor, String> properties = createBaseProperties();
         properties.put(SFTPTransfer.REMOTE_PATH, DIR_2);


[nifi] 05/47: NIFI-7201 - Update build to latest apache-maven-parent and split Github Actions builds to include OSX and distribute localization

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 211b0e8df41d8cd48de2d0d4c8a7e5de6b8711fc
Author: Joe Witt <jo...@apache.org>
AuthorDate: Tue Feb 25 15:29:50 2020 -0800

    NIFI-7201 - Update build to latest apache-maven-parent and split Github Actions builds to include OSX and distribute localization
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4091.
---
 .github/workflows/ci-workflow.yml                  | 145 ++++++++++++++++++---
 nifi-commons/nifi-security-utils/pom.xml           |   4 -
 .../nifi-elasticsearch-client-service/pom.xml      |   1 -
 .../nifi-framework/nifi-stateless/pom.xml          |   1 -
 pom.xml                                            |   7 +-
 5 files changed, 129 insertions(+), 29 deletions(-)

diff --git a/.github/workflows/ci-workflow.yml b/.github/workflows/ci-workflow.yml
index 54993c6..492663d 100644
--- a/.github/workflows/ci-workflow.yml
+++ b/.github/workflows/ci-workflow.yml
@@ -3,14 +3,11 @@ name: ci-workflow
 on: [push, pull_request]
 
 jobs:
-  ubuntu-build:
+  ubuntu-build-fr:
 
     timeout-minutes: 90
     runs-on: ubuntu-latest
-    strategy:
-      matrix:
-        java: [ '1.8', '11' ]
-    name: Ubuntu Build NiFi - JDK${{ matrix.java }}
+    name: Ubuntu Build NiFi - JDK 1.8 FR
 
     steps:
     - name: Checkout Code
@@ -19,49 +16,161 @@ jobs:
       uses: actions/cache@v1.1.2
       with:
         path: ~/.npm
-        key: linux-${{ matrix.java }}-npm-${{ hashFiles('**/package-lock.json') }}
+        key: linux-1.8-npm-${{ hashFiles('**/package-lock.json') }}
         restore-keys: |
-          linux-${{ matrix.java }}-npm-
+          linux-1.8-npm-
     - name: Check Maven Com Cache
       uses: actions/cache@v1.1.2
       with:
         path: ~/.m2/repository/com
-        key: linux-${{ matrix.java }}-maven-com-${{ hashFiles('**/pom.xml') }}
+        key: linux-1.8-maven-com-${{ hashFiles('**/pom.xml') }}
         restore-keys: |
-          linux-${{ matrix.java }}-maven-com-
+          linux-1.8-maven-com-
     - name: Check Maven Org Cache
       uses: actions/cache@v1.1.2
       with:
         path: ~/.m2/repository/org
-        key: linux-${{ matrix.java }}-maven-org-${{ hashFiles('**/pom.xml') }}
+        key: linux-1.8-maven-org-${{ hashFiles('**/pom.xml') }}
         restore-keys: |
-          linux-${{ matrix.java }}-maven-org-
+          linux-1.8-maven-org-
     - name: Check Maven Net Cache
       uses: actions/cache@v1.1.2
       with:
         path: ~/.m2/repository/net
-        key: linux-${{ matrix.java }}-maven-net-${{ hashFiles('**/pom.xml') }}
+        key: linux-1.8-maven-net-${{ hashFiles('**/pom.xml') }}
         restore-keys: |
-          linux-${{ matrix.java }}-maven-net-
+          linux-1.8-maven-net-
     - name: Check Maven IO Cache
       uses: actions/cache@v1.1.2
       with:
         path: ~/.m2/repository/io
-        key: linux-${{ matrix.java }}-maven-io-${{ hashFiles('**/pom.xml') }}
+        key: linux-1.8-maven-io-${{ hashFiles('**/pom.xml') }}
         restore-keys: |
-          linux-${{ matrix.java }}-maven-io-
-    - name: Set up JDK ${{ matrix.java }}
+          linux-1.8-maven-io-
+    - name: Set up JDK 1.8 FR
       uses: actions/setup-java@v1.3.0
       with:
-        java-version: ${{ matrix.java }}
+        java-version: 1.8
     - name: Build with Maven
       env:
-        MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN
+        MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=fr -Duser.region=FR"
       run: |
         mvn -version
         mvn -T 1C install -B -Pcontrib-check,include-grpc -Ddir-only -ntp -ff
         rm -rf ~/.m2/repository/org/apache/nifi
 
+  ubuntu-build-en:
+
+    timeout-minutes: 90
+    runs-on: ubuntu-latest
+    name: Ubuntu Build NiFi - JDK 11 EN
+
+    steps:
+      - name: Checkout Code
+        uses: actions/checkout@v2
+      - name: Check NPM Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.npm
+          key: linux-1.11-npm-${{ hashFiles('**/package-lock.json') }}
+          restore-keys: |
+            linux-1.11-npm-
+      - name: Check Maven Com Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/com
+          key: linux-1.11-maven-com-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.11-maven-com-
+      - name: Check Maven Org Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/org
+          key: linux-1.11-maven-org-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.11-maven-org-
+      - name: Check Maven Net Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/net
+          key: linux-1.11-maven-net-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.11-maven-net-
+      - name: Check Maven IO Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/io
+          key: linux-1.11-maven-io-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.11-maven-io-
+      - name: Set up JDK 11 EN
+        uses: actions/setup-java@v1.3.0
+        with:
+          java-version: 11
+      - name: Build with Maven
+        env:
+          MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=en -Duser.region=US"
+        run: |
+          mvn -version
+          mvn -T 1C install -B -Pcontrib-check,include-grpc -Ddir-only -ntp -ff
+          rm -rf ~/.m2/repository/org/apache/nifi
+
+  osx-build-jp:
+
+    timeout-minutes: 90
+    runs-on: macos-latest
+    name: MacOS Build NiFi - JDK 1.8 JP
+
+    steps:
+      - name: Checkout Code
+        uses: actions/checkout@v2
+      - name: Check NPM Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.npm
+          key: linux-1.8osx-npm-${{ hashFiles('**/package-lock.json') }}
+          restore-keys: |
+            linux-1.8osx-npm-
+      - name: Check Maven Com Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/com
+          key: linux-1.8osx-maven-com-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.8osx-maven-com-
+      - name: Check Maven Org Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/org
+          key: linux-1.8osx-maven-org-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.8osx-maven-org-
+      - name: Check Maven Net Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/net
+          key: linux-1.8osx-maven-net-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.8osx-maven-net-
+      - name: Check Maven IO Cache
+        uses: actions/cache@v1.1.2
+        with:
+          path: ~/.m2/repository/io
+          key: linux-1.8osx-maven-io-${{ hashFiles('**/pom.xml') }}
+          restore-keys: |
+            linux-1.8osx-maven-io-
+      - name: Set up JDK 1.8 JP
+        uses: actions/setup-java@v1.3.0
+        with:
+          java-version: 1.8
+      - name: Build with Maven
+        env:
+          MAVEN_OPTS: -Xmx2g -XX:ReservedCodeCacheSize=1g -XX:+UseG1GC -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN -Dmaven.surefire.arguments="-Duser.language=ja -Duser.region=JP"
+        run: |
+          mvn -version
+          mvn -T 1C install -B -Pcontrib-check,include-grpc -Ddir-only -ntp -ff
+          rm -rf ~/.m2/repository/org/apache/nifi
+
   windows-build:
 
     runs-on: windows-latest
diff --git a/nifi-commons/nifi-security-utils/pom.xml b/nifi-commons/nifi-security-utils/pom.xml
index ac755f4..9558d8e 100644
--- a/nifi-commons/nifi-security-utils/pom.xml
+++ b/nifi-commons/nifi-security-utils/pom.xml
@@ -37,10 +37,6 @@
             <artifactId>nifi-utils</artifactId>
             <version>1.11.4-SNAPSHOT</version>
         </dependency>
-        <!--<dependency>-->
-            <!--<groupId>org.slf4j</groupId>-->
-            <!--<artifactId>slf4j-api</artifactId>-->
-        <!--</dependency>-->
         <dependency>
             <groupId>ch.qos.logback</groupId>
             <artifactId>logback-classic</artifactId>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
index f383c9a..a02bcd8 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
@@ -87,7 +87,6 @@
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>log4j-over-slf4j</artifactId>
-            <version>${org.slf4j.version}</version>
         </dependency>
 
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
index bbc9209..1387a16 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
@@ -73,7 +73,6 @@
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
-            <version>1.7.25</version>
         </dependency>
 
         <!-- jackson dependencies -->
diff --git a/pom.xml b/pom.xml
index c0d4014..a0c2486 100644
--- a/pom.xml
+++ b/pom.xml
@@ -13,7 +13,7 @@
     <parent>
         <groupId>org.apache</groupId>
         <artifactId>apache</artifactId>
-        <version>22</version>
+        <version>23</version>
         <relativePath />
     </parent>
     <groupId>org.apache.nifi</groupId>
@@ -70,9 +70,6 @@
             <archive>https://mail-archives.apache.org/mod_mbox/nifi-commits</archive>
         </mailingList>
     </mailingLists>
-    <prerequisites>
-        <maven>${maven.min-version}</maven>
-    </prerequisites>
     <scm>
         <connection>scm:git:git://git.apache.org/nifi.git</connection>
         <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/nifi.git</developerConnection>
@@ -91,7 +88,7 @@
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <inceptionYear>2014</inceptionYear>
-        <org.slf4j.version>1.7.26</org.slf4j.version>
+        <org.slf4j.version>1.7.30</org.slf4j.version>
         <ranger.version>2.0.0</ranger.version>
         <jetty.version>9.4.19.v20190610</jetty.version>
         <jackson.version>2.9.10</jackson.version>


[nifi] 07/47: NIFI-6856 - Make client ID a non-required field for the MQTTConsume and MQTTProduce processors. Generates a random ID if not set.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 1d10b515bb5b6750e73aec623219ae8f5594a0c1
Author: Justin Miller <ju...@scanalyticsinc.com>
AuthorDate: Fri Nov 8 11:34:57 2019 -0600

    NIFI-6856 - Make client ID a non-required field for the MQTTConsume and MQTTProduce processors. Generates a
    random ID if not set.
    
    Also add group ID field to ConsumeMQTT processor. Allows consumer to join consumer group at $share/<group_id>/<topic_filter>
    
    add expression language support for the MQTT client ID
    
    Setting client id in publish test fails because it is not a flowfile attribute.
    Remove client id and autogenerate it when testing.
    
    Since the evaluation is done in onScheduled, there is no flow file available and we're not using the attributes to make the expression language evaluation. You can change the scope to use the Variable Registry.
    
    Co-Authored-By: Pierre Villard <pi...@gmail.com>
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #3879.
---
 .../apache/nifi/processors/mqtt/ConsumeMQTT.java   | 22 +++++++++++++++++++++-
 .../mqtt/common/AbstractMQTTProcessor.java         | 13 ++++++++++---
 .../nifi/processors/mqtt/TestPublishMQTT.java      |  1 -
 3 files changed, 31 insertions(+), 5 deletions(-)

diff --git a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java
index f0cba72..94d5397 100644
--- a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java
+++ b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java
@@ -90,6 +90,13 @@ public class ConsumeMQTT extends AbstractMQTTProcessor  implements MqttCallback
     public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
     public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
 
+    public static final PropertyDescriptor PROP_GROUPID = new PropertyDescriptor.Builder()
+            .name("Group ID")
+            .description("MQTT consumer group ID to use. If group ID not set, client will connect as individual consumer.")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
     public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
             .name("Topic Filter")
             .description("The MQTT topic filter to designate the topics to subscribe to.")
@@ -121,6 +128,7 @@ public class ConsumeMQTT extends AbstractMQTTProcessor  implements MqttCallback
     private volatile long maxQueueSize;
 
     private volatile int qos;
+    private volatile String topicPrefix = "";
     private volatile String topicFilter;
     private final AtomicBoolean scheduled = new AtomicBoolean(false);
 
@@ -136,6 +144,7 @@ public class ConsumeMQTT extends AbstractMQTTProcessor  implements MqttCallback
 
     static{
         final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
+        innerDescriptorsList.add(PROP_GROUPID);
         innerDescriptorsList.add(PROP_TOPIC_FILTER);
         innerDescriptorsList.add(PROP_QOS);
         innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
@@ -184,6 +193,12 @@ public class ConsumeMQTT extends AbstractMQTTProcessor  implements MqttCallback
                     .build());
         }
 
+        final boolean clientIDSet = context.getProperty(PROP_CLIENTID).isSet();
+        final boolean groupIDSet = context.getProperty(PROP_GROUPID).isSet();
+        if (clientIDSet && groupIDSet) {
+            results.add(new ValidationResult.Builder().subject("Client ID and Group ID").valid(false).explanation("if client ID is not unique, multiple nodes cannot join the consumer group").build());
+        }
+
         return results;
     }
 
@@ -208,6 +223,11 @@ public class ConsumeMQTT extends AbstractMQTTProcessor  implements MqttCallback
         qos = context.getProperty(PROP_QOS).asInteger();
         maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
         topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
+
+        if (context.getProperty(PROP_GROUPID).isSet()) {
+            topicPrefix = "$share/" + context.getProperty(PROP_GROUPID).getValue() + "/";
+        }
+
         scheduled.set(true);
     }
 
@@ -266,7 +286,7 @@ public class ConsumeMQTT extends AbstractMQTTProcessor  implements MqttCallback
             if (!mqttClient.isConnected()) {
                 logger.debug("Connecting client");
                 mqttClient.connect(connOpts);
-                mqttClient.subscribe(topicFilter, qos);
+                mqttClient.subscribe(topicPrefix + topicFilter, qos);
             }
         } catch (MqttException e) {
             logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
diff --git a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java
index a1e65f3..34c3e1f 100644
--- a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java
+++ b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java
@@ -23,6 +23,7 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -43,6 +44,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
+import java.util.UUID;
 
 import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
 import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
@@ -122,8 +124,9 @@ public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProces
 
     public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
             .name("Client ID")
-            .description("MQTT client ID to use")
-            .required(true)
+            .description("MQTT client ID to use. If not set, a UUID will be generated.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
 
@@ -297,7 +300,11 @@ public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProces
 
     protected void onScheduled(final ProcessContext context){
         broker = context.getProperty(PROP_BROKER_URI).getValue();
-        clientID = context.getProperty(PROP_CLIENTID).getValue();
+        clientID = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
+
+        if (clientID == null) {
+            clientID = UUID.randomUUID().toString();
+        }
 
         connOpts = new MqttConnectOptions();
         connOpts.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
diff --git a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/test/java/org/apache/nifi/processors/mqtt/TestPublishMQTT.java b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/test/java/org/apache/nifi/processors/mqtt/TestPublishMQTT.java
index 9916408..9c886d2 100644
--- a/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/test/java/org/apache/nifi/processors/mqtt/TestPublishMQTT.java
+++ b/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/test/java/org/apache/nifi/processors/mqtt/TestPublishMQTT.java
@@ -67,7 +67,6 @@ public class TestPublishMQTT extends TestPublishMqttCommon {
         UnitTestablePublishMqtt proc = new UnitTestablePublishMqtt();
         testRunner = TestRunners.newTestRunner(proc);
         testRunner.setProperty(PublishMQTT.PROP_BROKER_URI, "tcp://localhost:1883");
-        testRunner.setProperty(PublishMQTT.PROP_CLIENTID, "TestClient");
         testRunner.setProperty(PublishMQTT.PROP_RETAIN, "false");
         topic = "testTopic";
         testRunner.setProperty(PublishMQTT.PROP_TOPIC, topic);


[nifi] 43/47: NIFI-7208: Restore default timezone in JdbcCommon

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit d6655bd6b1c584e51f2fb104af0618e3d42fdc59
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Fri Mar 13 14:09:56 2020 -0400

    NIFI-7208: Restore default timezone in JdbcCommon
---
 .../src/main/java/org/apache/nifi/util/db/JdbcCommon.java     |  2 +-
 .../java/org/apache/nifi/processors/standard/TestPutSQL.java  | 11 +++++++----
 2 files changed, 8 insertions(+), 5 deletions(-)

diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
index 30ac518..10320e2 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java
@@ -788,7 +788,7 @@ public class JdbcCommon {
                     } else {
                         final DateTimeFormatter dtFormatter = getDateTimeFormatter(valueFormat);
                         LocalDateTime ldt = LocalDateTime.parse(parameterValue, dtFormatter);
-                        ts = Timestamp.from(ldt.atZone(ZoneId.of("UTC")).toInstant());
+                        ts = Timestamp.from(ldt.atZone(ZoneId.systemDefault()).toInstant());
                     }
 
                     stmt.setTimestamp(parameterIndex, ts);
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java
index 510e384..f7c1634 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutSQL.java
@@ -33,6 +33,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Time;
+import java.sql.Timestamp;
 import java.sql.Types;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
@@ -434,17 +435,19 @@ public class TestPutSQL {
         runner.enableControllerService(service);
         runner.setProperty(PutSQL.CONNECTION_POOL, "dbcp");
 
-        final String dateStr1 = "2002-02-02T12:02:02+00:00";
-        final long dateInt1 = 1012651322000L;
+        final String dateStr1 = "2002-02-02T12:02:02";
+        final String dateStrTimestamp1 = "2002-02-02 12:02:02";
+        final long dateInt1 = Timestamp.valueOf(dateStrTimestamp1).getTime();
 
         final String dateStr2 = "2002-02-02T12:02:02.123456789";
-        final long dateInt2 = 1012651322123L;
+        final String dateStrTimestamp2 = "2002-02-02 12:02:02.123456789";
+        final long dateInt2 = Timestamp.valueOf(dateStrTimestamp2).getTime();
         final long nanoInt2 = 123456789L;
 
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("sql.args.1.type", String.valueOf(Types.TIMESTAMP));
         attributes.put("sql.args.1.value", dateStr1);
-        attributes.put("sql.args.1.format", "ISO_OFFSET_DATE_TIME");
+        attributes.put("sql.args.1.format", "ISO_LOCAL_DATE_TIME");
         attributes.put("sql.args.2.type", String.valueOf(Types.TIMESTAMP));
         attributes.put("sql.args.2.value", dateStr2);
         attributes.put("sql.args.2.format", "yyyy-MM-dd'T'HH:mm:ss.SSSSSSSSS");


[nifi] 47/47: NIFI-7223 Resolved merge conflicts from additional test case for null key password.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 4e4fa38a179fafc9cd71467c6ba0705a5417b23e
Author: Andy LoPresto <al...@apache.org>
AuthorDate: Fri Mar 13 22:12:17 2020 -0700

    NIFI-7223 Resolved merge conflicts from additional test case for null key password.
    
    This closes #4145.
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../okhttp/OkHttpReplicationClientTest.groovy      | 24 ++++++++++++----------
 1 file changed, 13 insertions(+), 11 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
index c30c3e9..6c1d23d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
@@ -163,27 +163,29 @@ class OkHttpReplicationClientTest extends GroovyTestCase {
     }
 
     @Test
-    void testShouldUseKeystorePasswdIfKeyPasswdIsNull() {
+    void testShouldUseKeystorePasswordIfKeyPasswordIsNull() {
         // Arrange
         Map flowfileEncryptionProps = [
-                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
-                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+                (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
                 (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
-                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
-                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
-                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "passwordpassword",
-                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
-                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+                (NiFiProperties.SECURITY_KEYSTORE)         : "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : "passwordpassword",
+                (NiFiProperties.WEB_HTTPS_HOST)            : "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT)            : "51552",
         ]
         NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
 
         // Act
         OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+        logger.info("Created secure HTTPS client with TLS configured: ${client.isTLSConfigured()}")
 
         // Assert
-        assertNotNull(client.okHttpClient.sslSocketFactory)
-        assertEquals(SunX509KeyManagerImpl.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
-        assertNotNull(client.okHttpClient.sslSocketFactory.context.getX509KeyManager().credentialsMap["nifi-key"])    }
+        assert client.isTLSConfigured()
+        assert client.okHttpClient.sslSocketFactory
+        assert client.okHttpClient.sslSocketFactory.context.getX509KeyManager().credentialsMap["nifi-key"]
+    }
 
     @Test
     void testShouldFailIfKeyPasswordIsSetButKeystorePasswordIsBlank() {


[nifi] 04/47: NIFI-7164 Upgrade shyiko/mysql-binlog-connector-java dependency

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 98e4246bd2c1c6b9762789ac25ecf1a769688616
Author: Przemysław Dubaniewicz <18...@users.noreply.github.com>
AuthorDate: Mon Feb 24 13:53:23 2020 +0100

    NIFI-7164 Upgrade shyiko/mysql-binlog-connector-java dependency
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4076.
---
 .../nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml
index 9e4c2ee..fba978f 100644
--- a/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-cdc/nifi-cdc-mysql-bundle/nifi-cdc-mysql-processors/pom.xml
@@ -41,7 +41,7 @@ language governing permissions and limitations under the License. -->
         <dependency>
             <groupId>com.github.shyiko</groupId>
             <artifactId>mysql-binlog-connector-java</artifactId>
-            <version>0.11.0</version>
+            <version>0.20.1</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>


[nifi] 22/47: NIFI-7195 - Catch MongoException to route flow files to failure

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit f1d1c0f2e239776226b8261229e73ba41ce92a3c
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Tue Feb 25 14:00:31 2020 -0800

    NIFI-7195 - Catch MongoException to route flow files to failure
    
    This closes #4089
    
    Signed-off-by: Mike Thomsen <mt...@apache.org>
---
 .../main/java/org/apache/nifi/processors/mongodb/PutMongoRecord.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/PutMongoRecord.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/PutMongoRecord.java
index 5fbc819..3915733 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/PutMongoRecord.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/PutMongoRecord.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.mongodb;
 
+import com.mongodb.MongoException;
 import com.mongodb.WriteConcern;
 import com.mongodb.client.MongoCollection;
 import org.apache.nifi.annotation.behavior.EventDriven;
@@ -144,7 +145,7 @@ public class PutMongoRecord extends AbstractMongoProcessor {
             if (inserts.size() > 0) {
                 collection.insertMany(inserts);
             }
-        } catch (SchemaNotFoundException | IOException | MalformedRecordException e) {
+        } catch (SchemaNotFoundException | IOException | MalformedRecordException | MongoException e) {
             getLogger().error("PutMongoRecord failed with error:", e);
             session.transfer(flowFile, REL_FAILURE);
             error = true;


[nifi] 45/47: NIFI-7223 [WIP] Resolved compilation issues in unit test on OpenJDK 11 by removing Sun security class references. Added OkHttpReplicationClient#isTLSConfigured() method. Added unit test. NIFI-7223 Fixed remaining unit tests for TLS regression. Renamed tests for clarity.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 554a637b4055429649a70de785da3246530bde24
Author: Andy LoPresto <al...@apache.org>
AuthorDate: Fri Mar 13 20:47:21 2020 -0700

    NIFI-7223 [WIP] Resolved compilation issues in unit test on OpenJDK 11 by removing Sun security class references.
    Added OkHttpReplicationClient#isTLSConfigured() method.
    Added unit test.
    NIFI-7223 Fixed remaining unit tests for TLS regression.
    Renamed tests for clarity.
---
 .../okhttp/OkHttpReplicationClient.java            |  18 ++-
 .../okhttp/OkHttpReplicationClientTest.groovy      | 125 ++++++++++++++-------
 2 files changed, 99 insertions(+), 44 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
index ff63836..a123d81 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
@@ -17,6 +17,9 @@
 
 package org.apache.nifi.cluster.coordination.http.replication.okhttp;
 
+import static org.apache.nifi.security.util.SslContextFactory.ClientAuth.WANT;
+import static org.apache.nifi.security.util.SslContextFactory.createTrustSslContextWithTrustManagers;
+
 import com.fasterxml.jackson.annotation.JsonInclude.Include;
 import com.fasterxml.jackson.annotation.JsonInclude.Value;
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -71,9 +74,8 @@ import org.apache.nifi.util.Tuple;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.util.StreamUtils;
+
 // Using static imports because of the name conflict:
-import static org.apache.nifi.security.util.SslContextFactory.ClientAuth.WANT;
-import static org.apache.nifi.security.util.SslContextFactory.createTrustSslContextWithTrustManagers;
 
 public class OkHttpReplicationClient implements HttpReplicationClient {
     private static final Logger logger = LoggerFactory.getLogger(OkHttpReplicationClient.class);
@@ -84,6 +86,7 @@ public class OkHttpReplicationClient implements HttpReplicationClient {
 
     private final ObjectMapper jsonCodec = new ObjectMapper();
     private final OkHttpClient okHttpClient;
+    private boolean tlsConfigured = false;
 
     public OkHttpReplicationClient(final NiFiProperties properties) {
         jsonCodec.setDefaultPropertyInclusion(Value.construct(Include.NON_NULL, Include.ALWAYS));
@@ -150,6 +153,16 @@ public class OkHttpReplicationClient implements HttpReplicationClient {
         return response;
     }
 
+    /**
+     * Returns {@code true} if the client has TLS enabled and configured. Even clients created without explicit
+     * keystore and truststore values have a default cipher suite list available, but no keys to use.
+     *
+     * @return true if this client can present keys
+     */
+    public boolean isTLSConfigured() {
+        return tlsConfigured;
+    }
+
     private MultivaluedMap<String, String> getHeaders(final okhttp3.Response callResponse) {
         final Headers headers = callResponse.headers();
         final MultivaluedMap<String, String> headerMap = new MultivaluedHashMap<>();
@@ -319,6 +332,7 @@ public class OkHttpReplicationClient implements HttpReplicationClient {
         final Tuple<SSLSocketFactory, X509TrustManager> tuple = createSslSocketFactory(properties);
         if (tuple != null) {
             okHttpClientBuilder.sslSocketFactory(tuple.getKey(), tuple.getValue());
+            tlsConfigured = true;
         }
 
         return okHttpClientBuilder.build();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
index 712f80a..2cfb2cb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
@@ -26,8 +26,6 @@ import org.junit.runner.RunWith
 import org.junit.runners.JUnit4
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
-import sun.security.ssl.DummyX509KeyManager
-import sun.security.ssl.SunX509KeyManagerImpl
 
 @RunWith(JUnit4.class)
 class OkHttpReplicationClientTest extends GroovyTestCase {
@@ -139,75 +137,118 @@ class OkHttpReplicationClientTest extends GroovyTestCase {
     }
 
     @Test
-    void testShouldUseKeystorePasswdIfKeypasswdIsBlank() {
+    void testShouldUseKeystorePasswordIfKeyPasswordIsBlank() {
         // Arrange
-        Map flowfileEncryptionProps = [
-                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
-                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+        Map propsMap = [
+                (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
                 (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
-                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
-                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
-                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "passwordpassword",
-                (NiFiProperties.SECURITY_KEY_PASSWD): "",
-                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
-                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+                (NiFiProperties.SECURITY_KEYSTORE)         : "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : "passwordpassword",
+                (NiFiProperties.SECURITY_KEY_PASSWD)       : "",
+                (NiFiProperties.WEB_HTTPS_HOST)            : "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT)            : "51552",
         ]
-        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(propsMap))
 
         // Act
         OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+        logger.info("Created secure HTTPS client with TLS configured: ${client.isTLSConfigured()}")
 
         // Assert
-        assertNotNull(client.okHttpClient.sslSocketFactory)
-        assertEquals(SunX509KeyManagerImpl.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
-        assertNotNull(client.okHttpClient.sslSocketFactory.context.getX509KeyManager().credentialsMap["nifi-key"])
+        assert client.isTLSConfigured()
+        assert client.okHttpClient.sslSocketFactory
+        assert client.okHttpClient.sslSocketFactory.context.getX509KeyManager().credentialsMap["nifi-key"]
     }
 
     @Test
-    void testShouldFailIfKeyPasswdIsSetButKeystorePasswdIsBlank() {
+    void testShouldFailIfKeyPasswordIsSetButKeystorePasswordIsBlank() {
         // Arrange
-        Map flowfileEncryptionProps = [
-                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
-                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+        Map propsMap = [
+                (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
                 (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
-                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
-                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
-                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "",
-                (NiFiProperties.SECURITY_KEY_PASSWD): "passwordpassword",
-                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
-                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+                (NiFiProperties.SECURITY_KEYSTORE)         : "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : "",
+                (NiFiProperties.SECURITY_KEY_PASSWD)       : "passwordpassword",
+                (NiFiProperties.WEB_HTTPS_HOST)            : "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT)            : "51552",
         ]
-        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(propsMap))
 
         // Act
         OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+        logger.info("Created (invalid) secure HTTPS client with TLS configured: ${client.isTLSConfigured()}")
 
         // Assert
-        // The replication client will fail to initialize if the keystore password is missing, and will use
-        // a default empty DummyX509KeyManager instead. This is considered a failure to start the service.
-        assertSame(DummyX509KeyManager.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
+        assert !client.isTLSConfigured()
     }
 
     @Test
-    void testShouldFailIfKeyPasswdIsBlankAndKeystorePasswd() {
+    void testShouldFailIfKeyPasswordIsBlankAndKeystorePassword() {
         // Arrange
-        Map flowfileEncryptionProps = [
-                (NiFiProperties.SECURITY_TRUSTSTORE): "./src/test/resources/conf/truststore.jks",
-                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): "JKS",
+        Map propsMap = [
+                (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
                 (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
-                (NiFiProperties.SECURITY_KEYSTORE): "./src/test/resources/conf/keystore.jks",
-                (NiFiProperties.SECURITY_KEYSTORE_TYPE): "JKS",
-                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "",
-                (NiFiProperties.SECURITY_KEY_PASSWD): "",
-                (NiFiProperties.WEB_HTTPS_HOST): "localhost",
-                (NiFiProperties.WEB_HTTPS_PORT): "51552",
+                (NiFiProperties.SECURITY_KEYSTORE)         : "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : "",
+                (NiFiProperties.SECURITY_KEY_PASSWD)       : "",
+                (NiFiProperties.WEB_HTTPS_HOST)            : "localhost",
+                (NiFiProperties.WEB_HTTPS_PORT)            : "51552",
         ]
-        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(flowfileEncryptionProps))
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(propsMap))
 
         // Act
         OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+        logger.info("Created (invalid) secure HTTPS client with TLS configured: ${client.isTLSConfigured()}")
 
         // Assert
-        assertEquals(DummyX509KeyManager.class, client.okHttpClient.sslSocketFactory.context.getX509KeyManager().getClass())
+        assert !client.isTLSConfigured()
+    }
+
+    @Test
+    void testShouldDetermineIfTLSConfigured() {
+        // Arrange
+        Map propsMap = [(NiFiProperties.WEB_HTTPS_HOST): "localhost",
+                        (NiFiProperties.WEB_HTTPS_PORT): "51552",]
+
+        Map tlsPropsMap = [
+                (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
+                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_KEYSTORE)         : "./src/test/resources/conf/keystore.jks",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : "passwordpassword",
+                (NiFiProperties.SECURITY_KEY_PASSWD)       : "",
+        ] + propsMap
+
+
+        NiFiProperties mockNiFiProperties = new StandardNiFiProperties(new Properties(propsMap))
+        NiFiProperties mockTLSNiFiProperties = new StandardNiFiProperties(new Properties(tlsPropsMap))
+
+        // Remove the keystore password to create an invalid configuration
+        Map invalidTlsPropsMap = tlsPropsMap
+        invalidTlsPropsMap.remove(NiFiProperties.SECURITY_KEYSTORE_PASSWD)
+        NiFiProperties mockInvalidTLSNiFiProperties = new StandardNiFiProperties(new Properties(invalidTlsPropsMap))
+
+        // Act
+        OkHttpReplicationClient client = new OkHttpReplicationClient(mockNiFiProperties)
+        logger.info("Created plaintext HTTP client with TLS configured: ${client.isTLSConfigured()}")
+
+        OkHttpReplicationClient invalidTlsClient = new OkHttpReplicationClient(mockInvalidTLSNiFiProperties)
+        logger.info("Created (invalid) secure HTTPS client with TLS configured: ${invalidTlsClient.isTLSConfigured()}")
+
+        OkHttpReplicationClient tlsClient = new OkHttpReplicationClient(mockTLSNiFiProperties)
+        logger.info("Created secure HTTPS client with TLS configured: ${tlsClient.isTLSConfigured()}")
+
+
+        // Assert
+        assert !client.isTLSConfigured()
+        assert !invalidTlsClient.isTLSConfigured()
+        assert tlsClient.isTLSConfigured()
     }
 }


[nifi] 26/47: NIFI-4970 - Add a property to deal with empty GZIP HTTP response

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit aa2f5613e19ced75a43ffedeb0ff30ee70016111
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Tue Mar 10 15:46:06 2020 +0100

    NIFI-4970 - Add a property to deal with empty GZIP HTTP response
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4127.
---
 .../org/apache/nifi/processors/standard/InvokeHTTP.java  | 16 ++++++++++++++--
 .../apache/nifi/processors/standard/TestInvokeHTTP.java  |  1 +
 2 files changed, 15 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
index b5c853c..2ccaed5 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
@@ -23,6 +23,7 @@ import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
 import com.burgstaller.okhttp.digest.CachingAuthenticator;
 import com.burgstaller.okhttp.digest.DigestAuthenticator;
 import com.google.common.io.Files;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
@@ -408,6 +409,15 @@ public final class InvokeHTTP extends AbstractProcessor {
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
             .build();
 
+    public static final PropertyDescriptor IGNORE_RESPONSE_CONTENT = new PropertyDescriptor.Builder()
+            .name("ignore-response-content")
+            .description("If true, the processor will not write the response's content into the flow file.")
+            .displayName("Ignore response's content")
+            .required(true)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .build();
+
     private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
     public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
             = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
@@ -439,7 +449,8 @@ public final class InvokeHTTP extends AbstractProcessor {
             PROP_USE_CHUNKED_ENCODING,
             PROP_PENALIZE_NO_RETRY,
             PROP_USE_ETAG,
-            PROP_ETAG_MAX_CACHE_SIZE));
+            PROP_ETAG_MAX_CACHE_SIZE,
+            IGNORE_RESPONSE_CONTENT));
 
     // relationships
     public static final Relationship REL_SUCCESS_REQ = new Relationship.Builder()
@@ -486,6 +497,7 @@ public final class InvokeHTTP extends AbstractProcessor {
 
     private final AtomicReference<OkHttpClient> okHttpClientAtomicReference = new AtomicReference<>();
 
+    @Override
     protected void init(ProcessorInitializationContext context) {
         excludedHeaders.put("Trusted Hostname", "HTTP request header '{}' excluded. " +
                              "Update processor to use the SSLContextService instead. " +
@@ -842,7 +854,7 @@ public final class InvokeHTTP extends AbstractProcessor {
                 boolean outputBodyToRequestAttribute = (!isSuccess(statusCode) || putToAttribute) && requestFlowFile != null;
                 boolean outputBodyToResponseContent = (isSuccess(statusCode) && !putToAttribute) || context.getProperty(PROP_OUTPUT_RESPONSE_REGARDLESS).asBoolean();
                 ResponseBody responseBody = responseHttp.body();
-                boolean bodyExists = responseBody != null ? responseBody.contentLength() > 0 : false;
+                boolean bodyExists = responseBody != null && !context.getProperty(IGNORE_RESPONSE_CONTENT).asBoolean();
 
                 InputStream responseBodyStream = null;
                 SoftLimitBoundedByteArrayOutputStream outputStreamToRequestAttribute = null;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
index 1a9ee10..e25344c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
@@ -273,6 +273,7 @@ public class TestInvokeHTTP extends TestInvokeHttpCommon {
         addHandler(new EmptyGzipResponseHandler());
 
         runner.setProperty(InvokeHTTP.PROP_URL, url);
+        runner.setProperty(InvokeHTTP.IGNORE_RESPONSE_CONTENT, "true");
 
         createFlowFiles(runner);
 


[nifi] 20/47: NIFI-7197 - In-place replacement in LookupRecord processor

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit c82c1db628c4f8191fb965f1ada821c1bb3fbc55
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Tue Feb 25 08:53:36 2020 -0800

    NIFI-7197 - In-place replacement in LookupRecord processor
    
    This closes #4088
    
    Signed-off-by: Mark Payne <ma...@hotmail.com>
---
 .../nifi-standard-processors/pom.xml               |   2 +
 .../nifi/processors/standard/LookupRecord.java     | 126 ++++++++++--
 .../additionalDetails.html                         | 215 +++++++++++++++++++++
 .../nifi/processors/standard/TestLookupRecord.java |  89 +++++++++
 .../TestLookupRecord/lookup-array-input.json       |  29 +++
 .../TestLookupRecord/lookup-array-output.json      |   1 +
 6 files changed, 446 insertions(+), 16 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index 2cd98d6..896c85f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -581,6 +581,8 @@
                         <exclude>src/test/resources/TestValidateRecord/nested-map-schema.avsc</exclude>
                         <exclude>src/test/resources/TestValidateRecord/timestamp.avsc</exclude>
                         <exclude>src/test/resources/TestValidateRecord/timestamp.json</exclude>
+                        <exclude>src/test/resources/TestLookupRecord/lookup-array-input.json</exclude>
+                        <exclude>src/test/resources/TestLookupRecord/lookup-array-output.json</exclude>
                     </excludes>
                 </configuration>
             </plugin>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
index 23d1325..28705cc 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java
@@ -105,6 +105,14 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
     static final AllowableValue RESULT_RECORD_FIELDS = new AllowableValue("record-fields", "Insert Record Fields",
         "All of the fields in the Record that is retrieved from the Lookup Service will be inserted into the destination path.");
 
+    static final AllowableValue USE_PROPERTY = new AllowableValue("use-property", "Use Property",
+            "The \"Result RecordPath\" property will be used to determine which part of the record should be updated with the value returned by the Lookup Service");
+    static final AllowableValue REPLACE_EXISTING_VALUES = new AllowableValue("replace-existing-values", "Replace Existing Values",
+            "The \"Result RecordPath\" property will be ignored and the lookup service must be a single simple key lookup service. Every dynamic property value should "
+            + "be a record path. For each dynamic property, the value contained in the field corresponding to the record path will be used as the key in the Lookup "
+            + "Service and the value returned by the Lookup Service will be used to replace the existing value. It is possible to configure multiple dynamic properties "
+            + "to replace multiple values in one execution. This strategy only supports simple types replacements (strings, integers, etc).");
+
     static final PropertyDescriptor LOOKUP_SERVICE = new PropertyDescriptor.Builder()
         .name("lookup-service")
         .displayName("Lookup Service")
@@ -144,6 +152,16 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
         .required(true)
         .build();
 
+    static final PropertyDescriptor REPLACEMENT_STRATEGY = new PropertyDescriptor.Builder()
+        .name("record-update-strategy")
+        .displayName("Record Update Strategy")
+        .description("This property defines the strategy to use when updating the record with the value returned by the Lookup Service.")
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .allowableValues(REPLACE_EXISTING_VALUES, USE_PROPERTY)
+        .defaultValue(USE_PROPERTY.getValue())
+        .required(true)
+        .build();
+
     static final Relationship REL_MATCHED = new Relationship.Builder()
         .name("matched")
         .description("All records for which the lookup returns a value will be routed to this relationship")
@@ -182,6 +200,7 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
         properties.add(RESULT_RECORD_PATH);
         properties.add(ROUTING_STRATEGY);
         properties.add(RESULT_CONTENTS);
+        properties.add(REPLACEMENT_STRATEGY);
         return properties;
     }
 
@@ -214,24 +233,37 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
         }
 
         final Set<String> requiredKeys = validationContext.getProperty(LOOKUP_SERVICE).asControllerService(LookupService.class).getRequiredKeys();
-        final Set<String> missingKeys = requiredKeys.stream()
-            .filter(key -> !dynamicPropNames.contains(key))
-            .collect(Collectors.toSet());
 
-        if (!missingKeys.isEmpty()) {
-            final List<ValidationResult> validationResults = new ArrayList<>();
-            for (final String missingKey : missingKeys) {
-                final ValidationResult result = new ValidationResult.Builder()
-                    .subject(missingKey)
-                    .valid(false)
-                    .explanation("The configured Lookup Services requires that a key be provided with the name '" + missingKey
-                        + "'. Please add a new property to this Processor with a name '" + missingKey
-                        + "' and provide a RecordPath that can be used to retrieve the appropriate value.")
-                    .build();
-                validationResults.add(result);
+        if(validationContext.getProperty(REPLACEMENT_STRATEGY).getValue().equals(REPLACE_EXISTING_VALUES.getValue())) {
+            // it must be a single key lookup service
+            if(requiredKeys.size() != 1) {
+                return Collections.singleton(new ValidationResult.Builder()
+                        .subject(LOOKUP_SERVICE.getDisplayName())
+                        .valid(false)
+                        .explanation("When using \"" + REPLACE_EXISTING_VALUES.getDisplayName() + "\" as Record Update Strategy, "
+                                + "only a Lookup Service requiring a single key can be used.")
+                        .build());
+            }
+        } else {
+            final Set<String> missingKeys = requiredKeys.stream()
+                .filter(key -> !dynamicPropNames.contains(key))
+                .collect(Collectors.toSet());
+
+            if (!missingKeys.isEmpty()) {
+                final List<ValidationResult> validationResults = new ArrayList<>();
+                for (final String missingKey : missingKeys) {
+                    final ValidationResult result = new ValidationResult.Builder()
+                        .subject(missingKey)
+                        .valid(false)
+                        .explanation("The configured Lookup Services requires that a key be provided with the name '" + missingKey
+                            + "'. Please add a new property to this Processor with a name '" + missingKey
+                            + "' and provide a RecordPath that can be used to retrieve the appropriate value.")
+                        .build();
+                    validationResults.add(result);
+                }
+
+                return validationResults;
             }
-
-            return validationResults;
         }
 
         return Collections.emptyList();
@@ -263,6 +295,68 @@ public class LookupRecord extends AbstractRouteRecord<Tuple<Map<String, RecordPa
     protected Set<Relationship> route(final Record record, final RecordSchema writeSchema, final FlowFile flowFile, final ProcessContext context,
         final Tuple<Map<String, RecordPath>, RecordPath> flowFileContext) {
 
+        final boolean isInPlaceReplacement = context.getProperty(REPLACEMENT_STRATEGY).getValue().equals(REPLACE_EXISTING_VALUES.getValue());
+
+        if(isInPlaceReplacement) {
+            return doInPlaceReplacement(record, flowFile, context, flowFileContext);
+        } else {
+            return doResultPathReplacement(record, flowFile, context, flowFileContext);
+        }
+
+    }
+
+    private Set<Relationship> doInPlaceReplacement(Record record, FlowFile flowFile, ProcessContext context, Tuple<Map<String, RecordPath>, RecordPath> flowFileContext) {
+
+        final String lookupKey = (String) context.getProperty(LOOKUP_SERVICE).asControllerService(LookupService.class).getRequiredKeys().iterator().next();
+
+        final Map<String, RecordPath> recordPaths = flowFileContext.getKey();
+        final Map<String, Object> lookupCoordinates = new HashMap<>(recordPaths.size());
+
+        for (final Map.Entry<String, RecordPath> entry : recordPaths.entrySet()) {
+            final String coordinateKey = entry.getKey();
+            final RecordPath recordPath = entry.getValue();
+
+            final RecordPathResult pathResult = recordPath.evaluate(record);
+            final List<FieldValue> lookupFieldValues = pathResult.getSelectedFields()
+                .filter(fieldVal -> fieldVal.getValue() != null)
+                .collect(Collectors.toList());
+
+            if (lookupFieldValues.isEmpty()) {
+                final Set<Relationship> rels = routeToMatchedUnmatched ? UNMATCHED_COLLECTION : SUCCESS_COLLECTION;
+                getLogger().debug("RecordPath for property '{}' did not match any fields in a record for {}; routing record to {}", new Object[] {coordinateKey, flowFile, rels});
+                return rels;
+            }
+
+            for (FieldValue fieldValue : lookupFieldValues) {
+                final Object coordinateValue = (fieldValue.getValue() instanceof Number || fieldValue.getValue() instanceof Boolean)
+                        ? fieldValue.getValue() : DataTypeUtils.toString(fieldValue.getValue(), (String) null);
+                lookupCoordinates.put(lookupKey, coordinateValue);
+
+                final Optional<?> lookupValueOption;
+                try {
+                    lookupValueOption = lookupService.lookup(lookupCoordinates, flowFile.getAttributes());
+                } catch (final Exception e) {
+                    throw new ProcessException("Failed to lookup coordinates " + lookupCoordinates + " in Lookup Service", e);
+                }
+
+                if (!lookupValueOption.isPresent()) {
+                    final Set<Relationship> rels = routeToMatchedUnmatched ? UNMATCHED_COLLECTION : SUCCESS_COLLECTION;
+                    return rels;
+                }
+
+                final Object lookupValue = lookupValueOption.get();
+
+                final DataType inferredDataType = DataTypeUtils.inferDataType(lookupValue, RecordFieldType.STRING.getDataType());
+                fieldValue.updateValue(lookupValue, inferredDataType);
+
+            }
+        }
+
+        final Set<Relationship> rels = routeToMatchedUnmatched ? MATCHED_COLLECTION : SUCCESS_COLLECTION;
+        return rels;
+    }
+
+    private Set<Relationship> doResultPathReplacement(Record record, FlowFile flowFile, ProcessContext context, Tuple<Map<String, RecordPath>, RecordPath> flowFileContext) {
         final Map<String, RecordPath> recordPaths = flowFileContext.getKey();
         final Map<String, Object> lookupCoordinates = new HashMap<>(recordPaths.size());
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LookupRecord/additionalDetails.html b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LookupRecord/additionalDetails.html
new file mode 100644
index 0000000..df83708
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LookupRecord/additionalDetails.html
@@ -0,0 +1,215 @@
+<!DOCTYPE html>
+<html lang="en">
+    <!--
+      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.
+    -->
+    <head>
+        <meta charset="utf-8" />
+        <title>LookupRecord</title>
+
+        <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+    </head>
+
+    <body>
+    	<p>
+    		LookupRecord makes use of the NiFi <a href="../../../../../html/record-path-guide.html">
+    		RecordPath Domain-Specific Language (DSL)</a> to allow the user to indicate which field(s), 
+    		depending on the Record Update Strategy, in the Record should be updated. The Record will 
+    		be updated using the value returned by the provided Lookup Service.
+    	</p>
+
+		<h3>Record Update Strategy - Use Property</h3>
+		
+    	<p>
+    		In this case, the user should add, to the Processor's configuration, as much User-defined 
+    		Properties as required by the Lookup Service to form the lookup coordinates. The name of 
+    		the properties should match the names expected by the Lookup Service.
+    	</p>
+		
+    	<p>
+    		The field evaluated using the path configured in the "Result RecordPath" property will be 
+    		the field updated with the value returned by the Lookup Service.
+    	</p>
+    	
+    	<p>
+    		Let's assume a Simple Key Value Lookup Service containing the following key/value pairs:
+    	</p>
+    	
+<code>
+<pre>
+FR => France
+CA => Canada
+</pre>
+</code>
+		
+    	<p>
+    		Let's assume the following JSON with three records as input:
+    	</p>
+    	
+<code>
+<pre>
+[
+	{
+		"country": null,
+		"code": "FR"
+	}, {
+		"country": null,
+		"code": "CA"
+	}, {
+		"country": null,
+		"code": "JP"
+	}
+]
+</pre>
+</code>
+
+    	<p>
+    		The processor is configured with "Use Property" as "Record Update Strategy", the "Result 
+    		RecordPath" is configured with "/country" and a user-defined property is added with the 
+    		name "key" (as required by this Lookup Service) and the value "/code".
+    	</p>
+
+    	<p>
+    		When triggered, the processor will look for the value associated to the "/code" path and 
+    		will use the value as the "key" of the Lookup Service. The value returned by the Lookup 
+    		Service will be used to update the value corresponding to "/country". With the above 
+    		examples, it will produce:
+    	</p>
+
+<code>
+<pre>
+[
+	{
+		"country": "France",
+		"code": "FR"
+	}, {
+		"country": "Canada",
+		"code": "CA"
+	}, {
+		"country": null,
+		"code": "JP"
+	}
+]
+</pre>
+</code>
+
+		<h3>Record Update Strategy - Replace Existing Values</h3>
+
+    	<p>
+    		With this strategy, the "Result RecordPath" property will be ignored and the configured Lookup 
+    		Service must be a single single key lookup service. For each user-defined property, the value 
+    		contained in the field corresponding to the record path will be used as the key in the Lookup 
+    		Service and will be replaced by the value returned by the Lookup Service. It is possible to 
+    		configure multiple dynamic properties to update multiple fields in one execution. This strategy 
+    		only supports simple types replacements (strings, integers, etc).
+    	</p>
+
+    	<p>
+    		Since this strategy allows in-place replacement, it is possible to use Record Paths for fields 
+    		contained in arrays.
+    	</p>
+    	
+    	<p>
+    		Let's assume a Simple Key Value Lookup Service containing the following key/value pairs:
+    	</p>
+    	
+<code>
+<pre>
+FR => France
+CA => Canada
+fr => French
+en => English
+</pre>
+</code>
+		
+    	<p>
+    		Let's assume the following JSON with two records as input:
+    	</p>
+    	
+<code>
+<pre>
+[
+	{
+		"locales": [
+			{
+				"region": "FR",
+				"language": "fr"
+			}, {
+				"region": "US",
+				"language": "en"
+			}
+		]
+	}, {
+		"locales": [
+			{
+				"region": "CA",
+				"language": "fr"
+			}, 
+			{
+				"region": "JP",
+				"language": "ja"
+			}
+		]
+	}
+]
+</pre>
+</code>
+
+    	<p>
+    		The processor is configured with "Replace Existing Values" as "Record Update Strategy", 
+    		two user-defined properties are added: "region" => "/locales[*]/region" and "language 
+    		=> "/locales[*]/language"..
+    	</p>
+
+    	<p>
+    		When triggered, the processor will loop over the user-defined properties. First, it'll 
+    		search for the fields corresponding to "/locales[*]/region", for each value from the 
+    		record, the value will be used as the key with the Lookup Service and the value will 
+    		be replaced by the result returned by the Lookup Service. Example: the first region is 
+    		"FR" and this key is associated to the value "France" in the Lookup Service, so the 
+    		value "FR" is replaced by "France" in the record. With the above examples, it will 
+    		produce:
+    	</p>
+    	
+<code>
+<pre>
+[
+	{
+		"locales": [
+			{
+				"region": "France",
+				"language": "French"
+			}, {
+				"region": "US",
+				"language": "English"
+			}
+		]
+	}, {
+		"locales": [
+			{
+				"region": "Canada",
+				"language": "French"
+			}, 
+			{
+				"region": "JP",
+				"language": "ja"
+			}
+		]
+	}
+]
+</pre>
+</code>
+
+	</body>
+</html>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java
index f8fb158..86bba8a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestLookupRecord.java
@@ -19,9 +19,13 @@ package org.apache.nifi.processors.standard;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
 import org.apache.nifi.lookup.RecordLookupService;
 import org.apache.nifi.lookup.StringLookupService;
 import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.MockRecordParser;
@@ -37,6 +41,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -436,6 +442,88 @@ public class TestLookupRecord {
         out.assertContentEquals("John Doe,48,soccer,basketball\nJane Doe,47\n");
     }
 
+    @Test
+    public void testLookupArray() throws InitializationException, IOException {
+        TestRunner runner = TestRunners.newTestRunner(LookupRecord.class);
+        final MapLookup lookupService = new MapLookup();
+
+        final JsonTreeReader jsonReader = new JsonTreeReader();
+        runner.addControllerService("reader", jsonReader);
+        runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaInferenceUtil.INFER_SCHEMA);
+
+        final JsonRecordSetWriter jsonWriter = new JsonRecordSetWriter();
+        runner.addControllerService("writer", jsonWriter);
+        runner.setProperty(jsonWriter, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.INHERIT_RECORD_SCHEMA);
+
+        runner.addControllerService("reader", jsonReader);
+        runner.enableControllerService(jsonReader);
+        runner.addControllerService("writer", jsonWriter);
+        runner.enableControllerService(jsonWriter);
+        runner.addControllerService("lookup", lookupService);
+        runner.enableControllerService(lookupService);
+
+        runner.setProperty(LookupRecord.ROUTING_STRATEGY, LookupRecord.ROUTE_TO_SUCCESS);
+        runner.setProperty(LookupRecord.REPLACEMENT_STRATEGY, LookupRecord.REPLACE_EXISTING_VALUES);
+        runner.setProperty(LookupRecord.RECORD_READER, "reader");
+        runner.setProperty(LookupRecord.RECORD_WRITER, "writer");
+        runner.setProperty(LookupRecord.LOOKUP_SERVICE, "lookup");
+        runner.setProperty("lookupLanguage", "/locales[*]/language");
+        runner.setProperty("lookupRegion", "/locales[*]/region");
+        runner.setProperty("lookupFoo", "/foo/foo");
+
+        lookupService.addValue("FR", "France");
+        lookupService.addValue("CA", "Canada");
+        lookupService.addValue("fr", "French");
+        lookupService.addValue("key", "value");
+
+        runner.enqueue(new File("src/test/resources/TestLookupRecord/lookup-array-input.json").toPath());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(LookupRecord.REL_SUCCESS);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(LookupRecord.REL_SUCCESS).get(0);
+        out.assertContentEquals(new File("src/test/resources/TestLookupRecord/lookup-array-output.json").toPath());
+    }
+
+    @Test
+    public void testLookupArrayKeyNotInLRS() throws InitializationException, IOException {
+        TestRunner runner = TestRunners.newTestRunner(LookupRecord.class);
+        final MapLookup lookupService = new MapLookup();
+
+        final JsonTreeReader jsonReader = new JsonTreeReader();
+        runner.addControllerService("reader", jsonReader);
+        runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaInferenceUtil.INFER_SCHEMA);
+
+        final JsonRecordSetWriter jsonWriter = new JsonRecordSetWriter();
+        runner.addControllerService("writer", jsonWriter);
+        runner.setProperty(jsonWriter, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.INHERIT_RECORD_SCHEMA);
+
+        runner.addControllerService("reader", jsonReader);
+        runner.enableControllerService(jsonReader);
+        runner.addControllerService("writer", jsonWriter);
+        runner.enableControllerService(jsonWriter);
+        runner.addControllerService("lookup", lookupService);
+        runner.enableControllerService(lookupService);
+
+        runner.setProperty(LookupRecord.ROUTING_STRATEGY, LookupRecord.ROUTE_TO_MATCHED_UNMATCHED);
+        runner.setProperty(LookupRecord.REPLACEMENT_STRATEGY, LookupRecord.REPLACE_EXISTING_VALUES);
+        runner.setProperty(LookupRecord.RECORD_READER, "reader");
+        runner.setProperty(LookupRecord.RECORD_WRITER, "writer");
+        runner.setProperty(LookupRecord.LOOKUP_SERVICE, "lookup");
+        runner.setProperty("lookupLanguage", "/locales[*]/language");
+        runner.setProperty("lookupRegion", "/locales[*]/region");
+        runner.setProperty("lookupFoo", "/foo/foo");
+
+        lookupService.addValue("FR", "France");
+        lookupService.addValue("CA", "Canada");
+        lookupService.addValue("fr", "French");
+        lookupService.addValue("badkey", "value");
+
+        runner.enqueue(new File("src/test/resources/TestLookupRecord/lookup-array-input.json").toPath());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(LookupRecord.REL_UNMATCHED);
+    }
+
     private static class MapLookup extends AbstractControllerService implements StringLookupService {
         private final Map<String, String> values = new HashMap<>();
         private Map<String, Object> expectedContext;
@@ -449,6 +537,7 @@ public class TestLookupRecord {
             return String.class;
         }
 
+        @Override
         public Optional<String> lookup(final Map<String, Object> coordinates, Map<String, String> context) {
             validateContext(context);
             return lookup(coordinates);
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-input.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-input.json
new file mode 100644
index 0000000..f2902cd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-input.json
@@ -0,0 +1,29 @@
+[
+  {
+    "foo" : {
+      "foo" : "key"
+    },
+    "locales": [
+      {
+        "language" : "fr",
+        "region" : "CA"
+      }, {
+        "language" : "fr",
+        "region" : "FR"
+      }
+    ]
+  }, {
+    "foo" : {
+      "foo" : "key"
+    },
+    "locales": [
+      {
+        "language" : "fr",
+        "region" : "CA"
+      }, {
+        "language" : "fr",
+        "region" : "FR"
+      }
+    ]
+  }
+]
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-output.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-output.json
new file mode 100644
index 0000000..10169f8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestLookupRecord/lookup-array-output.json
@@ -0,0 +1 @@
+[{"foo":{"foo":"value"},"locales":[{"language":"French","region":"Canada"},{"language":"French","region":"France"}]},{"foo":{"foo":"value"},"locales":[{"language":"French","region":"Canada"},{"language":"French","region":"France"}]}]
\ No newline at end of file


[nifi] 23/47: Fiz writeBatch to avoid cycle in provenance graph

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit ae8e75222729b8dfaa3ae11c285a9bae072efe6e
Author: Eduardo Fontes <ed...@gmail.com>
AuthorDate: Fri Mar 6 16:06:21 2020 -0300

    Fiz writeBatch to avoid cycle in provenance graph
---
 .../org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
index 43c210c..df918d9 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
@@ -347,7 +347,9 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
         FlowFile flowFile = parent != null ? session.create(parent) : session.create();
         flowFile = session.importFrom(new ByteArrayInputStream(payload.getBytes(charset)), flowFile);
         flowFile = session.putAllAttributes(flowFile, extraAttributes);
-        session.getProvenanceReporter().receive(flowFile, getURI(context));
+        if (parent == null) {
+            session.getProvenanceReporter().receive(flowFile, getURI(context));
+        }
         session.transfer(flowFile, rel);
     }
 


[nifi] 38/47: NIFI-7251: Upgrade hadoop-client version to 3.2.1 to avoid the regression bug

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 42b453ca81ea22f93e27a12e61f20137fdbbc514
Author: Kotaro Terada <ko...@apache.org>
AuthorDate: Fri Mar 13 12:53:26 2020 +0900

    NIFI-7251: Upgrade hadoop-client version to 3.2.1 to avoid the regression bug
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #4141.
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index a0c2486..89cdb0a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -98,7 +98,7 @@
         <surefire.version>2.22.0</surefire.version>
         <!-- The Hadoop version used by nifi-hadoop-libraries-nar and any NARs that depend on it, other NARs that need
             a specific version should override this property, or use a more specific property like abc.hadoop.version -->
-        <hadoop.version>3.2.0</hadoop.version>
+        <hadoop.version>3.2.1</hadoop.version>
     </properties>
 
     <repositories>


[nifi] 24/47: NIFI-7210 - added PG path in bulletins for S2S Bulletin RT

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit a655be9857c7d8f9c25e47c23a87a98b1db7329b
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Thu Feb 27 14:59:03 2020 -0800

    NIFI-7210 - added PG path in bulletins for S2S Bulletin RT
    
    Added group path to BULLETIN table for QueryNiFiReportingTask
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #4100
---
 .../java/org/apache/nifi/reporting/Bulletin.java   |  9 ++++++
 .../org/apache/nifi/reporting/BulletinFactory.java | 15 ++++++++++
 .../org/apache/nifi/events/BulletinFactory.java    | 32 +++++++++++++++++++++-
 .../reporting/SiteToSiteBulletinReportingTask.java |  1 +
 .../additionalDetails.html                         |  1 +
 .../src/main/resources/schema-bulletins.avsc       |  1 +
 .../TestSiteToSiteBulletinReportingTask.java       |  5 ++--
 .../sql/bulletins/BulletinEnumerator.java          |  1 +
 .../reporting/sql/bulletins/BulletinTable.java     |  2 ++
 .../additionalDetails.html                         |  1 +
 10 files changed, 64 insertions(+), 4 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java b/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java
index 50684d6..3f160aa 100644
--- a/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java
+++ b/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java
@@ -33,6 +33,7 @@ public abstract class Bulletin implements Comparable<Bulletin> {
 
     private String groupId;
     private String groupName;
+    private String groupPath;
     private String sourceId;
     private String sourceName;
     private ComponentType sourceType;
@@ -98,6 +99,14 @@ public abstract class Bulletin implements Comparable<Bulletin> {
         this.groupName = groupName;
     }
 
+    public String getGroupPath() {
+        return groupPath;
+    }
+
+    public void setGroupPath(String groupPath) {
+        this.groupPath = groupPath;
+    }
+
     public String getSourceId() {
         return sourceId;
     }
diff --git a/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java b/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java
index 0208a2e..ca1ef6c 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java
@@ -53,4 +53,19 @@ public class BulletinFactory {
         bulletin.setMessage(message);
         return bulletin;
     }
+
+    public static Bulletin createBulletin(final String groupId, final String groupName, final String sourceId, final ComponentType sourceType,
+            final String sourceName, final String category, final String severity, final String message, final String groupPath) {
+        final Bulletin bulletin = new MockBulletin(currentId.getAndIncrement());
+        bulletin.setGroupId(groupId);
+        bulletin.setGroupName(groupName);
+        bulletin.setGroupPath(groupPath);
+        bulletin.setSourceId(sourceId);
+        bulletin.setSourceType(sourceType);
+        bulletin.setSourceName(sourceName);
+        bulletin.setCategory(category);
+        bulletin.setLevel(severity);
+        bulletin.setMessage(message);
+        return bulletin;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
index 76728ba..1d9f8cf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
@@ -49,7 +49,22 @@ public final class BulletinFactory {
         final ProcessGroup group = connectable.getProcessGroup();
         final String groupId = connectable.getProcessGroupIdentifier();
         final String groupName = group == null ? null : group.getName();
-        return BulletinFactory.createBulletin(groupId, groupName, connectable.getIdentifier(), type, connectable.getName(), category, severity, message);
+        final String groupPath = buildGroupPath(group);
+        return BulletinFactory.createBulletin(groupId, groupName, connectable.getIdentifier(), type, connectable.getName(), category, severity, message, groupPath);
+    }
+
+    private static String buildGroupPath(ProcessGroup group) {
+        if(group == null) {
+            return null;
+        } else {
+            String path = group.getName();
+            ProcessGroup parent = group.getParent();
+            while(parent != null) {
+                path = parent.getName() + " / " + path;
+                parent = parent.getParent();
+            }
+            return path;
+        }
     }
 
     public static Bulletin createBulletin(final String groupId, final String sourceId, final ComponentType sourceType, final String sourceName,
@@ -79,6 +94,21 @@ public final class BulletinFactory {
         return bulletin;
     }
 
+    public static Bulletin createBulletin(final String groupId, final String groupName, final String sourceId, final ComponentType sourceType,
+            final String sourceName, final String category, final String severity, final String message, final String groupPath) {
+        final Bulletin bulletin = new ComponentBulletin(currentId.getAndIncrement());
+        bulletin.setGroupId(groupId);
+        bulletin.setGroupName(groupName);
+        bulletin.setGroupPath(groupPath);
+        bulletin.setSourceId(sourceId);
+        bulletin.setSourceType(sourceType);
+        bulletin.setSourceName(sourceName);
+        bulletin.setCategory(category);
+        bulletin.setLevel(severity);
+        bulletin.setMessage(message);
+        return bulletin;
+    }
+
     public static Bulletin createBulletin(final String category, final String severity, final String message) {
         final Bulletin bulletin = new SystemBulletin(currentId.getAndIncrement());
         bulletin.setCategory(category);
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
index 1e68687..8fe220c 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
@@ -186,6 +186,7 @@ public class SiteToSiteBulletinReportingTask extends AbstractSiteToSiteReporting
         addField(builder, "bulletinCategory", bulletin.getCategory(), allowNullValues);
         addField(builder, "bulletinGroupId", bulletin.getGroupId(), allowNullValues);
         addField(builder, "bulletinGroupName", bulletin.getGroupName(), allowNullValues);
+        addField(builder, "bulletinGroupPath", bulletin.getGroupPath(), allowNullValues);
         addField(builder, "bulletinLevel", bulletin.getLevel(), allowNullValues);
         addField(builder, "bulletinMessage", bulletin.getMessage(), allowNullValues);
         addField(builder, "bulletinNodeAddress", bulletin.getNodeAddress(), allowNullValues);
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteBulletinReportingTask/additionalDetails.html b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteBulletinReportingTask/additionalDetails.html
index c76c138..13ced0b 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteBulletinReportingTask/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteBulletinReportingTask/additionalDetails.html
@@ -48,6 +48,7 @@
 	{ "name" : "bulletinCategory", "type" : ["string", "null"] },
 	{ "name" : "bulletinGroupId", "type" : ["string", "null"] },
 	{ "name" : "bulletinGroupName", "type" : ["string", "null"] },
+	{ "name" : "bulletinGroupPath", "type" : ["string", "null"] },
 	{ "name" : "bulletinLevel", "type" : ["string", "null"] },
 	{ "name" : "bulletinMessage", "type" : ["string", "null"] },
 	{ "name" : "bulletinNodeAddress", "type" : ["string", "null"] },
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/schema-bulletins.avsc b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/schema-bulletins.avsc
index 01b0f33..f681169 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/schema-bulletins.avsc
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/schema-bulletins.avsc
@@ -9,6 +9,7 @@
 	{ "name" : "bulletinCategory", "type" : ["string", "null"] },
 	{ "name" : "bulletinGroupId", "type" : ["string", "null"] },
 	{ "name" : "bulletinGroupName", "type" : ["string", "null"] },
+	{ "name" : "bulletinGroupPath", "type" : ["string", "null"] },
 	{ "name" : "bulletinLevel", "type" : ["string", "null"] },
 	{ "name" : "bulletinMessage", "type" : ["string", "null"] },
 	{ "name" : "bulletinNodeAddress", "type" : ["string", "null"] },
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteBulletinReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteBulletinReportingTask.java
index 9140cde..00cb4b9 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteBulletinReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteBulletinReportingTask.java
@@ -19,7 +19,6 @@ package org.apache.nifi.reporting;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.when;
 
@@ -82,7 +81,7 @@ public class TestSiteToSiteBulletinReportingTask {
     public void testSerializedForm() throws IOException, InitializationException {
         // creating the list of bulletins
         final List<Bulletin> bulletins = new ArrayList<Bulletin>();
-        bulletins.add(BulletinFactory.createBulletin("group-id", "group-name", "source-id", "source-name", "category", "severity", "message"));
+        bulletins.add(BulletinFactory.createBulletin("group-id", "group-name", "source-id", ComponentType.PROCESSOR, "source-name", "category", "severity", "message", "group-path"));
 
         // mock the access to the list of bulletins
         final ReportingContext context = Mockito.mock(ReportingContext.class);
@@ -125,7 +124,7 @@ public class TestSiteToSiteBulletinReportingTask {
         JsonObject bulletinJson = jsonReader.readArray().getJsonObject(0);
         assertEquals("message", bulletinJson.getString("bulletinMessage"));
         assertEquals("group-name", bulletinJson.getString("bulletinGroupName"));
-        assertNull(bulletinJson.get("bulletinSourceType"));
+        assertEquals("group-path", bulletinJson.getString("bulletinGroupPath"));
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinEnumerator.java b/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinEnumerator.java
index ac190bc..608f697 100644
--- a/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinEnumerator.java
+++ b/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinEnumerator.java
@@ -97,6 +97,7 @@ public class BulletinEnumerator implements Enumerator<Object> {
                 bulletin.getCategory(),
                 bulletin.getGroupId(),
                 bulletin.getGroupName(),
+                bulletin.getGroupPath(),
                 bulletin.getLevel(),
                 bulletin.getMessage(),
                 bulletin.getNodeAddress(),
diff --git a/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinTable.java b/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinTable.java
index 4fddb0a..3fc5ff6 100644
--- a/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinTable.java
+++ b/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/java/org/apache/nifi/reporting/sql/bulletins/BulletinTable.java
@@ -157,6 +157,7 @@ public class BulletinTable extends AbstractTable implements QueryableTable, Tran
                 "bulletinCategory",
                 "bulletinGroupId",
                 "bulletinGroupName",
+                "bulletinGroupPath",
                 "bulletinLevel",
                 "bulletinMessage",
                 "bulletinNodeAddress",
@@ -178,6 +179,7 @@ public class BulletinTable extends AbstractTable implements QueryableTable, Tran
                 typeFactory.createJavaType(String.class),
                 typeFactory.createJavaType(String.class),
                 typeFactory.createJavaType(String.class),
+                typeFactory.createJavaType(String.class),
                 typeFactory.createJavaType(Date.class)
         );
 
diff --git a/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.sql.QueryNiFiReportingTask/additionalDetails.html b/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.sql.QueryNiFiReportingTask/additionalDetails.html
index 7b55eef..a56324d 100644
--- a/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.sql.QueryNiFiReportingTask/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-sql-reporting-bundle/nifi-sql-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.sql.QueryNiFiReportingTask/additionalDetails.html
@@ -102,6 +102,7 @@
     <tr><td>bulletinCategory</td><td>String</td></tr>
     <tr><td>bulletinGroupId</td><td>String</td></tr>
     <tr><td>bulletinGroupName</td><td>String</td></tr>
+    <tr><td>bulletinGroupPath</td><td>String</td></tr>
     <tr><td>bulletinLevel</td><td>String</td></tr>
     <tr><td>bulletinMessage</td><td>String</td></tr>
     <tr><td>bulletinNodeAddress</td><td>String</td></tr>


[nifi] 37/47: NIFI-6742 Use JUnit TemporaryFolder when creating test databases

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 85efc0998e3a34e4850be4afd02a65de52876557
Author: Adam Taft <mi...@ngc.com>
AuthorDate: Wed Oct 23 21:03:38 2019 -0600

    NIFI-6742 Use JUnit TemporaryFolder when creating test databases
    
    - Add @Rule for TemporaryFolder
    - Replace use of previous target/db with TemporaryFolder/db
    - Remove use of ~/test db (in home directory)
    - Remove System.out lines
    
    Signed-off-by: Marc Parisi <ph...@apache.org>
    
    This closes #4137.
---
 .../java/org/apache/nifi/dbcp/DBCPServiceTest.java | 88 ++++++----------------
 1 file changed, 25 insertions(+), 63 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/DBCPServiceTest.java b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/DBCPServiceTest.java
index 7522125..fdad692 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/DBCPServiceTest.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/DBCPServiceTest.java
@@ -23,13 +23,16 @@ import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.h2.tools.Server;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
+import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetAddress;
 import java.net.MalformedURLException;
@@ -51,13 +54,21 @@ import static org.junit.Assert.assertTrue;
 
 public class DBCPServiceTest {
 
-    final static String DB_LOCATION = "target/db";
+    @Rule
+    public TemporaryFolder tempFolder = new TemporaryFolder(new File("target"));
+
+    private String dbLocation;
 
     @BeforeClass
     public static void setup() {
         System.setProperty("derby.stream.error.file", "target/derby.log");
     }
 
+    @Before
+    public void before() throws IOException {
+        this.dbLocation = new File(tempFolder.getRoot(), "db").getPath();
+    }
+
     /**
      * Missing property values.
      */
@@ -79,12 +90,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-good1", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
@@ -103,12 +110,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-good1", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
@@ -129,12 +132,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-good1", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
@@ -153,12 +152,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-good1", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
@@ -191,12 +186,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-good1", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
@@ -260,12 +251,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-good1", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
@@ -329,12 +316,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-exhaust", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
 
@@ -367,7 +350,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-dropcreate", service);
 
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:h2:tcp://localhost:" + server.getPort() + "/~/test");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL,
+            "jdbc:h2:tcp://localhost:" + server.getPort() + "/./" + dbLocation);
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.h2.Driver");
         runner.enableControllerService(service);
 
@@ -378,7 +362,6 @@ public class DBCPServiceTest {
         // get and verify connections
         for (int i = 0; i < 10; i++) {
             final Connection connection = dbcpService.getConnection();
-            System.out.println(connection);
             Assert.assertNotNull(connection);
             assertValidConnectionH2(connection, i);
             connection.close();
@@ -391,7 +374,6 @@ public class DBCPServiceTest {
 
         for (int i = 0; i < 10; i++) {
             final Connection connection = dbcpService.getConnection();
-            System.out.println(connection);
             Assert.assertNotNull(connection);
             assertValidConnectionH2(connection, i);
             connection.close();
@@ -415,7 +397,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-dropcreate", service);
 
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:h2:tcp://localhost:" + server.getPort() + "/~/test");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL,
+            "jdbc:h2:tcp://localhost:" + server.getPort() + "/./" + dbLocation);
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.h2.Driver");
         runner.setProperty(service, DBCPConnectionPool.VALIDATION_QUERY, "SELECT 5");
         runner.enableControllerService(service);
@@ -427,7 +410,6 @@ public class DBCPServiceTest {
         // get and verify connections
         for (int i = 0; i < 10; i++) {
             final Connection connection = dbcpService.getConnection();
-            System.out.println(connection);
             Assert.assertNotNull(connection);
             assertValidConnectionH2(connection, i);
             connection.close();
@@ -443,7 +425,6 @@ public class DBCPServiceTest {
         // Pool should remove invalid connections and create new valid connections.
         for (int i = 0; i < 10; i++) {
             final Connection connection = dbcpService.getConnection();
-            System.out.println(connection);
             Assert.assertNotNull(connection);
             assertValidConnectionH2(connection, i);
             connection.close();
@@ -472,15 +453,9 @@ public class DBCPServiceTest {
     @Test
     public void testDropInvalidConnectionsDerby() throws Exception {
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-        if (dbLocation.exists())
-            throw new RuntimeException("Still exists " + dbLocation.getAbsolutePath());
-
         // Start Derby server.
         System.setProperty("derby.drda.startNetworkServer", "true");
-        System.setProperty("derby.system.home", DB_LOCATION);
+        System.setProperty("derby.system.home", dbLocation);
         NetworkServerControl serverControl = new NetworkServerControl(InetAddress.getLocalHost(),1527);
         serverControl.start(new PrintWriter(System.out, true));
 
@@ -506,26 +481,17 @@ public class DBCPServiceTest {
 
         for (int i = 0; i < 10; i++) {
             final Connection connection = dbcpService.getConnection();
-            System.out.println(connection);
             Assert.assertNotNull(connection);
             assertValidConnectionDerby(connection, i);
             connection.close();
         }
 
         serverControl.shutdown();
-        dbLocation.delete();
-        if (dbLocation.exists())
-            throw new RuntimeException("Still exists " + dbLocation.getAbsolutePath());
-        try {
-            serverControl.ping();
-        } catch (Exception e) {
-        }
 
         Thread.sleep(2000);
 
         for (int i = 0; i < 10; i++) {
             final Connection connection = dbcpService.getConnection();
-            System.out.println(connection);
             Assert.assertNotNull(connection);
             assertValidConnectionDerby(connection, i);
             connection.close();
@@ -553,12 +519,8 @@ public class DBCPServiceTest {
         final DBCPConnectionPool service = new DBCPConnectionPool();
         runner.addControllerService("test-exhaust", service);
 
-        // remove previous test database, if any
-        final File dbLocation = new File(DB_LOCATION);
-        dbLocation.delete();
-
         // set embedded Derby database connection url
-        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
+        runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + dbLocation + ";create=true");
         runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
         runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
         runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");


[nifi] 30/47: unit test reproducing the issue

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 450bcd02a2d49c00f91b9607c1f1dee5d430b632
Author: Pierre Villard <pi...@gmail.com>
AuthorDate: Fri Mar 6 15:28:08 2020 +0100

    unit test reproducing the issue
---
 .../nifi-standard-processors/pom.xml               |  2 +
 .../processors/standard/TestValidateRecord.java    | 46 ++++++++++++++++++++++
 .../TestValidateRecord/int-maps-data.json          | 27 +++++++++++++
 .../TestValidateRecord/int-maps-schema.avsc        | 44 +++++++++++++++++++++
 .../nifi/json/AbstractJsonRowRecordReader.java     |  4 +-
 5 files changed, 121 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index 896c85f..089dd01 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -583,6 +583,8 @@
                         <exclude>src/test/resources/TestValidateRecord/timestamp.json</exclude>
                         <exclude>src/test/resources/TestLookupRecord/lookup-array-input.json</exclude>
                         <exclude>src/test/resources/TestLookupRecord/lookup-array-output.json</exclude>
+                        <exclude>src/test/resources/TestValidateRecord/int-maps-schema.avsc</exclude>
+                        <exclude>src/test/resources/TestValidateRecord/int-maps-data.json</exclude>
                     </excludes>
                 </configuration>
             </plugin>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
index f53dd5c..1b4b7e4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
@@ -18,6 +18,8 @@
 package org.apache.nifi.processors.standard;
 
 import org.apache.nifi.avro.AvroReader;
+import org.apache.nifi.avro.AvroReaderWithEmbeddedSchema;
+import org.apache.nifi.avro.AvroRecordReader;
 import org.apache.nifi.avro.AvroRecordSetWriter;
 import org.apache.nifi.csv.CSVReader;
 import org.apache.nifi.csv.CSVRecordSetWriter;
@@ -31,6 +33,7 @@ import org.apache.nifi.schema.inference.SchemaInferenceUtil;
 import org.apache.nifi.serialization.DateTimeUtils;
 import org.apache.nifi.serialization.MalformedRecordException;
 import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.MockRecordWriter;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
@@ -45,10 +48,12 @@ import org.junit.Test;
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.UnsupportedEncodingException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
+import java.util.HashMap;
 import java.util.Optional;
 
 import static org.junit.Assert.assertEquals;
@@ -547,4 +552,45 @@ public class TestValidateRecord {
         final MockFlowFile validFlowFileInferredSchema = runner.getFlowFilesForRelationship(ValidateRecord.REL_VALID).get(0);
         validFlowFileInferredSchema.assertContentEquals(new File("src/test/resources/TestValidateRecord/timestamp.json"));
     }
+
+    @Test
+    public void testValidateMaps() throws IOException, InitializationException, MalformedRecordException {
+        final String validateSchema = new String(Files.readAllBytes(Paths.get("src/test/resources/TestValidateRecord/int-maps-schema.avsc")), StandardCharsets.UTF_8);
+
+        final JsonTreeReader jsonReader = new JsonTreeReader();
+        runner.addControllerService("reader", jsonReader);
+        runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, "schema-text-property");
+        runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_TEXT, validateSchema);
+        runner.enableControllerService(jsonReader);
+
+        final AvroRecordSetWriter avroWriter = new AvroRecordSetWriter();
+        runner.addControllerService("writer", avroWriter);
+        runner.enableControllerService(avroWriter);
+
+        runner.setProperty(ValidateRecord.RECORD_READER, "reader");
+        runner.setProperty(ValidateRecord.RECORD_WRITER, "writer");
+        runner.setProperty(ValidateRecord.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY);
+        runner.setProperty(ValidateRecord.SCHEMA_TEXT, validateSchema);
+        runner.setProperty(ValidateRecord.INVALID_RECORD_WRITER, "writer");
+        runner.setProperty(ValidateRecord.ALLOW_EXTRA_FIELDS, "false");
+
+        runner.enqueue(Paths.get("src/test/resources/TestValidateRecord/int-maps-data.json"));
+        runner.run();
+
+        runner.assertTransferCount(ValidateRecord.REL_VALID, 1);
+        final MockFlowFile validFlowFile = runner.getFlowFilesForRelationship(ValidateRecord.REL_VALID).get(0);
+
+        byte[] source = validFlowFile.toByteArray();
+
+        try (final InputStream in = new ByteArrayInputStream(source); final AvroRecordReader reader = new AvroReaderWithEmbeddedSchema(in)) {
+            final Object[] values = reader.nextRecord().getValues();
+            assertEquals("uuid", values[0]);
+            assertEquals(2, ((HashMap<?,?>) values[1]).size());
+            final Object[] data = (Object[]) values[2];
+            assertEquals(2, ( (HashMap<?,?>) ((MapRecord) data[0]).getValue("points")).size());
+            assertEquals(2, ( (HashMap<?,?>) ((MapRecord) data[1]).getValue("points")).size());
+            assertEquals(2, ( (HashMap<?,?>) ((MapRecord) data[2]).getValue("points")).size());
+        }
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-data.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-data.json
new file mode 100644
index 0000000..00f9d13
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-data.json
@@ -0,0 +1,27 @@
+{
+  "id": "uuid",
+  "points": {
+    "1": 0,
+    "2": 0
+  },
+  "data": [
+    {
+      "points": {
+        "3": 0,
+        "4": 0
+      }
+    },
+    {
+      "points": {
+        "5": 0,
+        "6": 0
+      }
+    },
+    {
+      "points": {
+        "7": 0,
+        "8": 0
+      }
+    }
+  ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-schema.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-schema.avsc
new file mode 100644
index 0000000..9aa302a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/int-maps-schema.avsc
@@ -0,0 +1,44 @@
+{
+  "name": "statistic",
+  "type": "record",
+  "fields": [
+    {
+      "name": "id",
+      "type": "string"
+    },
+    {
+      "name": "points",
+      "type": {
+        "type": "map",
+        "values": "int"
+      }
+    },
+    {
+      "name": "data",
+      "type": {
+        "type": "array",
+        "items": {
+          "name": "data",
+          "type": "record",
+          "fields": [
+            {
+              "name": "version",
+              "type": [
+                "null",
+                "string"
+              ],
+              "default": null
+            },
+            {
+              "name": "points",
+              "type": {
+                "type": "map",
+                "values": "int"
+              }
+            }
+          ]
+        }
+      }
+    }
+  ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
index 69b7fab..f163707 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java
@@ -160,9 +160,9 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
                     } catch (final Exception e) {
                         return textValue;
                     }
+                default:
+                    return textValue;
             }
-
-            return textValue;
         }
 
         if (fieldNode.isArray()) {