[nifi] 02/02: NIFI-6403 and NIFI-6404: Elasticsearch 7 support
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/nifi.git commit 124cdbd3fe5446bff08346e5a28f2883f13e2848 Author: Chris Sampson AuthorDate: Mon Nov 16 13:03:50 2020 + NIFI-6403 and NIFI-6404: Elasticsearch 7 support Addressed PR#4153 comments; removed ES Version property and made Type optional in all ES HTTP/Record processors, applying sensible default values where required; use _source queyr parameter instead of _source_include/s as it's compatible between ES versions Fix unit test compilation to use JDK8-compatible library/method Better optional type and id handling for PutElasticsearchRecord; update nifi-elasticsearch-client-service build dependencies to use latest versions of Elasticsearch in each supported major version (5/6/7); addressed several warnings in ElasticSearchClientServiceImpl This closes #4667. Signed-off-by: Koji Kawamura --- .../nifi-elasticsearch-client-service/pom.xml | 16 +-- .../ElasticSearchClientServiceImpl.java| 124 ++--- .../AbstractElasticsearchHttpProcessor.java| 66 +-- .../elasticsearch/FetchElasticsearchHttp.java | 22 ++-- .../elasticsearch/PutElasticsearchHttp.java| 9 +- .../elasticsearch/PutElasticsearchHttpRecord.java | 9 +- .../elasticsearch/QueryElasticsearchHttp.java | 27 ++--- .../elasticsearch/ScrollElasticsearchHttp.java | 26 ++--- .../elasticsearch/ITQueryElasticsearchHttp.java| 3 - .../elasticsearch/ITScrollElasticsearchHttp.java | 2 - .../PutElasticsearchHttpRecordIT.java | 6 - .../elasticsearch/TestFetchElasticsearchHttp.java | 87 --- .../elasticsearch/TestPutElasticsearchHttp.java| 15 +-- .../TestPutElasticsearchHttpRecord.java| 10 +- .../elasticsearch/TestQueryElasticsearchHttp.java | 50 ++--- .../elasticsearch/TestScrollElasticsearchHttp.java | 37 ++ .../PutElasticsearchRecordTest.groovy | 37 ++ 17 files changed, 225 insertions(+), 321 deletions(-) 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 7d05609..baf8f7a 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 @@ -26,7 +26,7 @@ jar -5.6.15 +5.6.16 setup-5.script faketype @@ -71,7 +71,7 @@ commons-io commons-io -2.6 +2.8.0 @@ -82,7 +82,7 @@ org.apache.commons commons-lang3 -3.9 +3.11 org.slf4j @@ -146,7 +146,7 @@ org.apache.httpcomponents httpclient -4.5.10 +4.5.13 org.apache.nifi @@ -176,7 +176,7 @@ integration-6 -6.7.1 +6.8.13 _doc setup-6.script @@ -184,7 +184,7 @@ integration-7 -7.0.0 +7.10.0 setup-7.script _doc @@ -196,7 +196,7 @@ org.apache.maven.plugins maven-failsafe-plugin -3.0.0-M3 +3.0.0-M5 ${type.name} @@ -206,7 +206,7 @@ com.github.alexcojocaru elasticsearch-maven-plugin -6.13 +6.19 testCluster 9500 diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java index 3a686af..bad70ba 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java @@ -26,7 +26,6 @@ import java.net.URL; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import
[nifi] branch main updated (857eeca -> 124cdbd)
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a change to branch main in repository https://gitbox.apache.org/repos/asf/nifi.git. from 857eeca NIFI-8032: fix record-path-guide.adoc new a3d845a NIFI-6403: Adding Elasticsearch7 support to HTTP processors new 124cdbd NIFI-6403 and NIFI-6404: Elasticsearch 7 support The 2 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: .../nifi-elasticsearch-client-service/pom.xml | 16 +-- .../ElasticSearchClientServiceImpl.java| 124 ++--- .../AbstractElasticsearchHttpProcessor.java| 36 +++--- .../elasticsearch/FetchElasticsearchHttp.java | 11 +- .../elasticsearch/PutElasticsearchHttp.java| 5 +- .../elasticsearch/PutElasticsearchHttpRecord.java | 5 +- .../elasticsearch/QueryElasticsearchHttp.java | 17 ++- .../elasticsearch/ScrollElasticsearchHttp.java | 13 +-- .../PutElasticsearchHttpRecordIT.java | 1 + .../elasticsearch/TestFetchElasticsearchHttp.java | 83 +- .../elasticsearch/TestPutElasticsearchHttp.java| 34 +- .../TestPutElasticsearchHttpRecord.java| 113 ++- .../elasticsearch/TestQueryElasticsearchHttp.java | 28 - .../elasticsearch/TestScrollElasticsearchHttp.java | 29 - .../PutElasticsearchRecordTest.groovy | 37 ++ 15 files changed, 405 insertions(+), 147 deletions(-)
[nifi] 01/02: NIFI-6403: Adding Elasticsearch7 support to HTTP processors
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/nifi.git commit a3d845a38f93fed1f65ec85c851ca845ef186556 Author: Joe Gresock AuthorDate: Thu Mar 19 17:17:04 2020 + NIFI-6403: Adding Elasticsearch7 support to HTTP processors This closes #4153. Signed-off-by: Koji Kawamura --- .../AbstractElasticsearchHttpProcessor.java| 84 +-- .../elasticsearch/FetchElasticsearchHttp.java | 19 ++-- .../elasticsearch/PutElasticsearchHttp.java| 6 +- .../elasticsearch/PutElasticsearchHttpRecord.java | 6 +- .../elasticsearch/QueryElasticsearchHttp.java | 20 ++-- .../elasticsearch/ScrollElasticsearchHttp.java | 23 ++-- .../elasticsearch/ITQueryElasticsearchHttp.java| 3 + .../elasticsearch/ITScrollElasticsearchHttp.java | 2 + .../PutElasticsearchHttpRecordIT.java | 7 ++ .../elasticsearch/TestFetchElasticsearchHttp.java | 16 ++- .../elasticsearch/TestPutElasticsearchHttp.java| 39 ++- .../TestPutElasticsearchHttpRecord.java| 117 - .../elasticsearch/TestQueryElasticsearchHttp.java | 64 ++- .../elasticsearch/TestScrollElasticsearchHttp.java | 50 - 14 files changed, 405 insertions(+), 51 deletions(-) diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java index ab211c4..291d99b 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java @@ -36,9 +36,11 @@ import okhttp3.RequestBody; import okhttp3.Response; import okhttp3.Route; import org.apache.commons.text.StringEscapeUtils; +import org.apache.nifi.components.AllowableValue; 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.logging.ComponentLog; import org.apache.nifi.processor.ProcessContext; @@ -55,8 +57,13 @@ import org.apache.nifi.util.StringUtils; * A base class for Elasticsearch processors that use the HTTP API */ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElasticsearchProcessor { +enum ElasticsearchVersion { +ES_7, +ES_LESS_THAN_7 +} static final String FIELD_INCLUDE_QUERY_PARAM = "_source_include"; +static final String FIELD_INCLUDE_QUERY_PARAM_ES7 = "_source_includes"; static final String QUERY_QUERY_PARAM = "q"; static final String SORT_QUERY_PARAM = "sort"; static final String SIZE_QUERY_PARAM = "size"; @@ -127,6 +134,18 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .build(); +public static final PropertyDescriptor ES_VERSION = new PropertyDescriptor.Builder() +.name("elasticsearch-http-version") +.displayName("Elasticsearch Version") +.description("The major version of elasticsearch (this affects some HTTP query parameters and the way responses are parsed).") +.required(true) +.allowableValues( +new AllowableValue(ElasticsearchVersion.ES_LESS_THAN_7.name(), "< 7.0", "Any version of Elasticsearch less than 7.0"), +new AllowableValue(ElasticsearchVersion.ES_7.name(), "7.x", "Elasticsearch version 7.x")) +.defaultValue(ElasticsearchVersion.ES_LESS_THAN_7.name()) +.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.build(); + private final AtomicReference okHttpClientAtomicReference = new AtomicReference<>(); @Override @@ -148,6 +167,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic static { final List properties = new ArrayList<>(); properties.add(ES_URL); +properties.add(ES_VERSION); properties.add(PROP_SSL_CONTEXT_SERVICE); properties.add(CHARSET); properties.add(USERNAME); @@ -287,9 +307,12 @@ public abstract class AbstractEl
[nifi] branch master updated: NIFI-7348 Wait - Removes WAIT_START_TIMESTAMP after expiration
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 8e3f420 NIFI-7348 Wait - Removes WAIT_START_TIMESTAMP after expiration 8e3f420 is described below commit 8e3f42051fb3c7da27873de8b6d4507827a7b80d Author: EndzeitBegins <1115+endzeitbeg...@users.noreply.github.com> AuthorDate: Mon Apr 13 16:19:55 2020 +0200 NIFI-7348 Wait - Removes WAIT_START_TIMESTAMP after expiration This closes #4201. Signed-off-by: Koji Kawamura --- .../org/apache/nifi/processors/standard/Wait.java | 4 +- .../apache/nifi/processors/standard/TestWait.java | 62 -- 2 files changed, 37 insertions(+), 29 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java index 45ffcb2..37f4479 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java @@ -87,7 +87,7 @@ import static org.apache.nifi.processor.FlowFileFilter.FlowFileFilterResult.REJE @WritesAttributes({ @WritesAttribute(attribute = "wait.start.timestamp", description = "All FlowFiles will have an attribute 'wait.start.timestamp', which sets the " + "initial epoch timestamp when the file first entered this processor. This is used to determine the expiration time of the FlowFile. " -+ "This attribute is not written when the FlowFile is transferred to failure or success"), ++ "This attribute is not written when the FlowFile is transferred to failure, expired or success"), @WritesAttribute(attribute = "wait.counter.", description = "If a signal exists when the processor runs, " + "each count value in the signal is copied.") }) @@ -375,7 +375,7 @@ public class Wait extends AbstractProcessor { final Relationship finalRelationship = relationship; final List flowFilesWithSignalAttributes = routedFlowFiles.getValue().stream() .map(f -> { -if (REL_SUCCESS.equals(finalRelationship)) { +if (REL_SUCCESS.equals(finalRelationship) || REL_EXPIRED.equals(finalRelationship)) { // These flowFiles will be exiting the wait, clear the timer f = clearWaitState(session, f); } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestWait.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestWait.java index 7970601..d8fffdc 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestWait.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestWait.java @@ -69,8 +69,8 @@ public class TestWait { // no cache key attribute runner.assertAllFlowFilesTransferred(Wait.REL_WAIT, 1); -// timestamp must be present - runner.getFlowFilesForRelationship(Wait.REL_WAIT).get(0).assertAttributeExists(Wait.WAIT_START_TIMESTAMP); +MockFlowFile ff = runner.getFlowFilesForRelationship(Wait.REL_WAIT).get(0); +ff.assertAttributeExists(Wait.WAIT_START_TIMESTAMP); // timestamp must be set runner.clearTransferState(); } @@ -103,7 +103,7 @@ public class TestWait { runner.assertAllFlowFilesTransferred(Wait.REL_WAIT, 1); MockFlowFile ff = runner.getFlowFilesForRelationship(Wait.REL_WAIT).get(0); -ff.assertAttributeExists(Wait.WAIT_START_TIMESTAMP); +ff.assertAttributeExists(Wait.WAIT_START_TIMESTAMP); // timestamp must be set runner.clearTransferState(); runner.enqueue(ff); @@ -112,6 +112,8 @@ public class TestWait { runner.run(); runner.assertAllFlowFilesTransferred(Wait.REL_EXPIRED, 1); +ff = runner.getFlowFilesForRelationship(Wait.REL_EXPIRED).get(0); +ff.assertAttributeNotExists(Wait.WAIT_START_TIMESTAMP); // timestamp must be cleared runner.clearTransferState(); } @@ -129,7 +131,7 @@ public class TestWait { runner.assertAllFlowFilesTransferred(Wait.REL_WAIT, 1); MockFlowFile ff = runner.getFlowFilesForRelationship(Wait.REL_WAIT).get(0); -
[nifi] branch master updated: NIFI-4012 Azure Event Hub UI typos and cleanup
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new c72a561 NIFI-4012 Azure Event Hub UI typos and cleanup c72a561 is described below commit c72a5618c0d61cbd2001e0453b402b8e3ec6cd90 Author: Shayne Burgess AuthorDate: Wed Sep 18 13:54:59 2019 -0700 NIFI-4012 Azure Event Hub UI typos and cleanup This closes #3749. Signed-off-by: Koji Kawamura --- .../azure/eventhub/ConsumeAzureEventHub.java | 52 +++--- .../azure/eventhub/GetAzureEventHub.java | 34 +++--- .../azure/eventhub/PutAzureEventHub.java | 14 +++--- 3 files changed, 52 insertions(+), 48 deletions(-) diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java index 800f26d..f1e3a64 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/ConsumeAzureEventHub.java @@ -78,15 +78,15 @@ import java.util.concurrent.TimeUnit; import static org.apache.nifi.util.StringUtils.isEmpty; @Tags({"azure", "microsoft", "cloud", "eventhub", "events", "streaming", "streams"}) -@CapabilityDescription("Receives messages from a Microsoft Azure Event Hub, writing the contents of the Azure message to the content of the FlowFile.") +@CapabilityDescription("Receives messages from Azure Event Hubs, writing the contents of the message to the content of the FlowFile.") @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) @TriggerSerially @WritesAttributes({ -@WritesAttribute(attribute = "eventhub.enqueued.timestamp", description = "The time (in milliseconds since epoch, UTC) at which the message was enqueued in the Azure Event Hub"), +@WritesAttribute(attribute = "eventhub.enqueued.timestamp", description = "The time (in milliseconds since epoch, UTC) at which the message was enqueued in the event hub"), @WritesAttribute(attribute = "eventhub.offset", description = "The offset into the partition at which the message was stored"), -@WritesAttribute(attribute = "eventhub.sequence", description = "The Azure Sequence number associated with the message"), -@WritesAttribute(attribute = "eventhub.name", description = "The name of the Event Hub from which the message was pulled"), -@WritesAttribute(attribute = "eventhub.partition", description = "The name of the Azure Partition from which the message was pulled") +@WritesAttribute(attribute = "eventhub.sequence", description = "The sequence number associated with the message"), +@WritesAttribute(attribute = "eventhub.name", description = "The name of the event hub from which the message was pulled"), +@WritesAttribute(attribute = "eventhub.partition", description = "The name of the partition from which the message was pulled") }) public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor { @@ -95,7 +95,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor { static final PropertyDescriptor NAMESPACE = new PropertyDescriptor.Builder() .name("event-hub-namespace") .displayName("Event Hub Namespace") -.description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to -ns.") +.description("The namespace that the Azure Event Hubs is assigned to. This is generally equal to -ns.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .required(true) @@ -103,7 +103,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor { static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder() .name("event-hub-name") .displayName("Event Hub Name") -.description("The name of the Azure Event Hub to pull messages from.") +.description("The name of the event hub to pull messages from.") .addValidator(StandardValidators.NON_EMPTY_VALIDATO
[nifi] branch master updated: NIFI-5929 Support for IBM MQ multi-instance queue managers
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 1dfbc97 NIFI-5929 Support for IBM MQ multi-instance queue managers 1dfbc97 is described below commit 1dfbc97c074a5fc5c8e68124e0984504cfa97813 Author: Veli Kerim Celik AuthorDate: Fri Jan 4 17:10:00 2019 +0100 NIFI-5929 Support for IBM MQ multi-instance queue managers proper line break more proper line break :) Link to external javadocs and add some code comments Test that properties is set on ConnectionFactory cleanup made two static properties optional and elaborated on 'Additional Details' page minor corrections to user doc open external links in new tab Do broker accommodations only if property is set. Add fallback accommodation. fix test Add support for colon pair(s) for IBM MQ. In fallback broker handling use first pair if multiple given. This closes #3246. Signed-off-by: Koji Kawamura --- .../nifi/jms/cf/JMSConnectionFactoryProvider.java | 129 +++--- .../additionalDetails.html | 199 +++--- .../cf/JMSConnectionFactoryProviderForTest.java| 49 +++ .../jms/cf/JMSConnectionFactoryProviderTest.java | 437 +++-- 4 files changed, 685 insertions(+), 129 deletions(-) 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 781ce65..b831162 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 @@ -17,10 +17,12 @@ 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; @@ -39,7 +41,6 @@ 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.reporting.InitializationException; import org.apache.nifi.ssl.SSLContextService; import org.apache.nifi.ssl.SSLContextService.ClientAuth; import org.slf4j.Logger; @@ -51,12 +52,10 @@ import org.slf4j.LoggerFactory; * * 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 JMSConnectionFactoryProviderDefinition#CLIENT_LIB_DIR_PATH}, allowing - * it then to create an instance of the target {@link ConnectionFactory} based - * on the provided - * {@link JMSConnectionFactoryProviderDefinition#CONNECTION_FACTORY_IMPL} which - * can be than access via {@link #getConnectionFactory()} method. - * + * {@link #CLIENT_LIB_DIR_PATH}, 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 #getConnectionFactory()} method. */ @Tags({"jms", "messaging", "integration", "queue", "topic", "publish", "subscribe"}) @CapabilityDescription("Provides a generic service to create vendor specific javax.jms.ConnectionFactory implementations. " @@ -91,12 +90,12 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl .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)") +.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.") ++ "ConnectionFactory implementation. Required if target is not ActiveMQ.") .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator())) -.required(true) +.required(
[nifi] branch master updated (af0777b -> 7f96fa1)
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from af0777b NIFI-6834 Encoding attachment filename add 7f96fa1 NIFI-6395: Thread-safety bug fixed and added new flag property to handle the way to adjust the counters No new revisions were added by this update. Summary of changes: .../apache/nifi/processors/standard/CountText.java | 59 +- 1 file changed, 34 insertions(+), 25 deletions(-)
[nifi] branch master updated: NIFI-6395: Thread-safety bug fixed and added new flag property to handle the way to adjust the counters
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 7f96fa1 NIFI-6395: Thread-safety bug fixed and added new flag property to handle the way to adjust the counters 7f96fa1 is described below commit 7f96fa1d0dbcfd061fba1e061c891ac35da4333d Author: Ivan Ezequiel Rodriguez AuthorDate: Wed Jun 26 02:10:13 2019 -0300 NIFI-6395: Thread-safety bug fixed and added new flag property to handle the way to adjust the counters Update CountText.java Local variable changes by AtomicInteger NIFI-6395 - Fix line is longer than 200 characters This closes #3552. Signed-off-by: Koji Kawamura --- .../apache/nifi/processors/standard/CountText.java | 59 +- 1 file changed, 34 insertions(+), 25 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CountText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CountText.java index 20195bd..d624580 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CountText.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CountText.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.nifi.annotation.behavior.EventDriven; @@ -141,7 +142,16 @@ public class CountText extends AbstractProcessor { .allowableValues(getStandardCharsetNames()) .defaultValue(StandardCharsets.UTF_8.displayName()) .build(); - +public static final PropertyDescriptor ADJUST_IMMEDIATELY = new PropertyDescriptor.Builder() +.name("ajust-immediately") +.displayName("Call Immediate Adjustment") +.description("If true, the counter will be updated immediately, without regard to whether the ProcessSession is commit or rolled back;" + +"otherwise, the counter will be incremented only if and when the ProcessSession is committed.") +.required(true) +.allowableValues("true", "false") +.defaultValue("false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.build(); private static Set getStandardCharsetNames() { return STANDARD_CHARSETS.stream().map(c -> c.displayName()).collect(Collectors.toSet()); } @@ -164,7 +174,8 @@ public class CountText extends AbstractProcessor { TEXT_WORD_COUNT_PD, TEXT_CHARACTER_COUNT_PD, SPLIT_WORDS_ON_SYMBOLS_PD, -CHARACTER_ENCODING_PD)); +CHARACTER_ENCODING_PD, +ADJUST_IMMEDIATELY)); relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE))); @@ -175,13 +186,9 @@ public class CountText extends AbstractProcessor { private volatile boolean countWords; private volatile boolean countCharacters; private volatile boolean splitWordsOnSymbols; +private volatile boolean adjustImmediately; private volatile String characterEncoding = StandardCharsets.UTF_8.name(); -private volatile int lineCount; -private volatile int lineNonEmptyCount; -private volatile int wordCount; -private volatile int characterCount; - @Override public Set getRelationships() { return relationships; @@ -199,6 +206,8 @@ public class CountText extends AbstractProcessor { ? context.getProperty(TEXT_CHARACTER_COUNT_PD).asBoolean() : false; this.splitWordsOnSymbols = context.getProperty(SPLIT_WORDS_ON_SYMBOLS_PD).isSet() ? context.getProperty(SPLIT_WORDS_ON_SYMBOLS_PD).asBoolean() : false; +this.adjustImmediately = context.getProperty(ADJUST_IMMEDIATELY).isSet() +? context.getProperty(ADJUST_IMMEDIATELY).asBoolean() : false; this.characterEncoding = context.getProperty(CHARACTER_ENCODING_PD).getValue(); } @@ -213,10 +222,10 @@ public class CountText extends AbstractProcessor { } AtomicBoolean error = new AtomicBoolean(); -lineCount = 0; -lineNonEmptyCount = 0; -wordCount = 0; -characterCount = 0; +final AtomicInteger lineCount = new AtomicInteger(0); +final AtomicInteger lineNonEmptyCount = new AtomicInteger(0)
[nifi] branch master updated: [NIFI-6374] ConsumeEWS fails when email attachment has no content type - ensure non null content type
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 08fe648 [NIFI-6374] ConsumeEWS fails when email attachment has no content type - ensure non null content type 08fe648 is described below commit 08fe648b1c61ba38d6332661219ebeaf67c3486b Author: JF Beauvais AuthorDate: Mon Jul 22 10:36:26 2019 +0200 [NIFI-6374] ConsumeEWS fails when email attachment has no content type - ensure non null content type This closes #3596. Signed-off-by: Koji Kawamura --- .../src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java index 462f405..aeeb5ee 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ConsumeEWS.java @@ -454,7 +454,8 @@ public class ConsumeEWS extends AbstractProcessor { FileAttachment file = (FileAttachment)x; file.load(); -ByteArrayDataSource bds = new ByteArrayDataSource(file.getContent(), file.getContentType()); +String type = file.getContentType() == null ? "text/plain" : file.getContentType(); +ByteArrayDataSource bds = new ByteArrayDataSource(file.getContent(), type); mm.attach(bds,file.getName(), "", EmailAttachment.ATTACHMENT); } catch (MessagingException e) {
[nifi] branch master updated: NIFI-6275 ListHDFS now ignores scheme and authority when uses "Full Path" filter mode Updated description for "Full Path" filter mode to state that it will ignore scheme
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 8d74822 NIFI-6275 ListHDFS now ignores scheme and authority when uses "Full Path" filter mode Updated description for "Full Path" filter mode to state that it will ignore scheme and authority Added tests to TestListHDFS for listing an empty and nonexistent dirs Updated TestListHDFS' mock file system to track state properly when FileStatus instances are added, and updated listStatus to work properly with the underlying Map that contains FileStatus instances Updated ListHDFS' addi [...] 8d74822 is described below commit 8d748223ff8f80c7a85fc38013ecf0b221adc2da Author: Jeff Storck AuthorDate: Mon May 13 16:26:36 2019 -0400 NIFI-6275 ListHDFS now ignores scheme and authority when uses "Full Path" filter mode Updated description for "Full Path" filter mode to state that it will ignore scheme and authority Added tests to TestListHDFS for listing an empty and nonexistent dirs Updated TestListHDFS' mock file system to track state properly when FileStatus instances are added, and updated listStatus to work properly with the underlying Map that contains FileStatus instances Updated ListHDFS' additional details to document "Full Path" filter mode ignoring scheme and authority, with an example Updated TestRunners, StandardProcessorTestRunner, MockProcessorInitializationContext to support passing in a logger. NIFI-6275 Updated the "Full Path" filter mode to check the full path of a file with and without its scheme and authority against the filter regex Added additional documentation for how ListHDFS handles scheme and authority when "Full Path" filter mode is used Added test case for "Full Path" filter mode with a regular expression that includes scheme and authority This closes #3483. Signed-off-by: Koji Kawamura --- .../util/MockProcessorInitializationContext.java | 6 +- .../nifi/util/StandardProcessorTestRunner.java | 8 +- .../java/org/apache/nifi/util/TestRunners.java | 50 + .../apache/nifi/processors/hadoop/ListHDFS.java| 35 ++- .../additionalDetails.html | 11 +- .../nifi/processors/hadoop/TestListHDFS.java | 243 +++-- 6 files changed, 316 insertions(+), 37 deletions(-) diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java index e44e731..6a26371 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java @@ -33,8 +33,12 @@ public class MockProcessorInitializationContext implements ProcessorInitializati private final MockProcessContext context; public MockProcessorInitializationContext(final Processor processor, final MockProcessContext context) { +this(processor, context, null); +} + +public MockProcessorInitializationContext(final Processor processor, final MockProcessContext context, final MockComponentLog logger) { processorId = UUID.randomUUID().toString(); -logger = new MockComponentLog(processorId, processor); +this.logger = logger == null ? new MockComponentLog(processorId, processor) : logger; this.context = context; } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 89af696..d995f8e 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -99,6 +99,10 @@ public class StandardProcessorTestRunner implements TestRunner { } StandardProcessorTestRunner(final Processor processor, String processorName) { +this(processor, processorName, null); +} + +StandardProcessorTestRunner(final Processor processor, String processorName, MockComponentLog logger) { this.processor = processor; this.idGenerator = new AtomicLong(0L); this.sharedState = new SharedSessionState(processor, idGenerator); @@ -108,9 +112,9 @@ public class StandardProcessorTestRunner implements TestRunner { this.variableRegistry = new MockVariableRegistry(); this.context = new MockProcessContext(processor, processorName, processorStateManager, variableRegistry); -final MockProcessorInitializationContext mockInitContext = new MockProcessorInitializationContext(processor, c
[nifi] branch master updated: NIFI-6613: If LengthDelimitedJournal gets poisoned, log the reason and hold onto it so that it can be included as the cause of subsequent Exceptions that are thrown
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 6b17c4b NIFI-6613: If LengthDelimitedJournal gets poisoned, log the reason and hold onto it so that it can be included as the cause of subsequent Exceptions that are thrown 6b17c4b is described below commit 6b17c4b1347d91177bdece540b3485e962e30a2b Author: Mark Payne AuthorDate: Tue Sep 3 19:05:00 2019 -0400 NIFI-6613: If LengthDelimitedJournal gets poisoned, log the reason and hold onto it so that it can be included as the cause of subsequent Exceptions that are thrown This closes #3704. Signed-off-by: Koji Kawamura --- .../org/apache/nifi/wali/LengthDelimitedJournal.java | 20 ++-- 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java index e0dcb63..c3ed0b5 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java @@ -74,7 +74,7 @@ public class LengthDelimitedJournal implements WriteAheadJournal { private int transactionCount; private boolean headerWritten = false; -private volatile boolean poisoned = false; +private volatile Throwable poisonCause = null; private volatile boolean closed = false; private final ByteBuffer transactionPreamble = ByteBuffer.allocate(12); // guarded by synchronized block @@ -142,7 +142,11 @@ public class LengthDelimitedJournal implements WriteAheadJournal { @Override public synchronized boolean isHealthy() { -return !closed && !poisoned; +return !closed && !isPoisoned(); +} + +private boolean isPoisoned() { +return poisonCause != null; } @Override @@ -342,10 +346,12 @@ public class LengthDelimitedJournal implements WriteAheadJournal { private void checkState() throws IOException { -if (poisoned) { +final Throwable cause = this.poisonCause; +if (cause != null) { +logger.debug("Cannot update Write Ahead Log because the log has already been poisoned", cause); throw new IOException("Cannot update journal file " + journalFile + " because this journal has already encountered a failure when attempting to write to the file. " + "If the repository is able to checkpoint, then this problem will resolve itself. However, if the repository is unable to be checkpointed " -+ "(for example, due to being out of storage space or having too many open files), then this issue may require manual intervention."); ++ "(for example, due to being out of storage space or having too many open files), then this issue may require manual intervention.", cause); } if (closed) { @@ -354,7 +360,9 @@ public class LengthDelimitedJournal implements WriteAheadJournal { } protected void poison(final Throwable t) { -this.poisoned = true; +this.poisonCause = t; + +logger.error("Marking Write-Ahead journal file {} as poisoned due to {}", journalFile, t, t); try { if (fileOut != null) { @@ -390,7 +398,7 @@ public class LengthDelimitedJournal implements WriteAheadJournal { try { if (fileOut != null) { -if (!poisoned) { +if (!isPoisoned()) { fileOut.write(JOURNAL_COMPLETE); }
[nifi] branch master updated: NIFI-6596 Moves AmazonS3EncryptionService interface to `nifi-aws-service-api` package.
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 93e6f19 NIFI-6596 Moves AmazonS3EncryptionService interface to `nifi-aws-service-api` package. 93e6f19 is described below commit 93e6f195d939ea0187f48f306a750ee41ac82de8 Author: Troy Melhase AuthorDate: Wed Sep 4 13:10:47 2019 -0800 NIFI-6596 Moves AmazonS3EncryptionService interface to `nifi-aws-service-api` package. This closes #3694. Signed-off-by: Koji Kawamura --- nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml | 4 .../org/apache/nifi/processors/aws/s3/AmazonS3EncryptionService.java | 0 2 files changed, 4 insertions(+) diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml index 936b36c..ebc3410 100644 --- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml +++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/pom.xml @@ -39,6 +39,10 @@ +com.amazonaws +aws-java-sdk-s3 + + org.apache.nifi nifi-api diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AmazonS3EncryptionService.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/src/main/java/org/apache/nifi/processors/aws/s3/AmazonS3EncryptionService.java similarity index 100% rename from nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AmazonS3EncryptionService.java rename to nifi-nar-bundles/nifi-aws-bundle/nifi-aws-service-api/src/main/java/org/apache/nifi/processors/aws/s3/AmazonS3EncryptionService.java
[nifi] branch master updated: NIFI-6530 - HTTP SiteToSite server returns 201 in case no data is available
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new f01668e NIFI-6530 - HTTP SiteToSite server returns 201 in case no data is available f01668e is described below commit f01668e66ad2e45197915769e966a4be27e1592e Author: Arpad Boda AuthorDate: Mon Aug 12 17:30:30 2019 +0200 NIFI-6530 - HTTP SiteToSite server returns 201 in case no data is available This closes #3647. Signed-off-by: Koji Kawamura --- .../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 | 15 -- .../apache/nifi/web/api/DataTransferResource.java | 19 ++- .../stateless/core/StatelessRemoteOutputPort.java | 8 ++- .../apache/nifi/toolkit/s2s/SiteToSiteCliMain.java | 3 ++ .../nifi/toolkit/s2s/SiteToSiteReceiver.java | 4 ++ 18 files changed, 253 insertions(+), 73 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 a443967..14c163b 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 @@ -262,7 +262,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 PeerStatus getNextPeerStatus(final TransferDirection direction) { +public ArrayList getPeerStatuses(final TransferDirection direction) { List peerList = peerStatuses; if (isPeerRefreshNeeded(peerList)) { peerRefreshLock.lock(); @@ -289,25 +289,15 @@ public class PeerSelector { } } + if (peerList == null || peerList.isEmpty()) { -return null; +return new ArrayList(); } -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; -} -} +ArrayList retVal = new ArrayList<>(peerList); +retVal.removeIf(p -> isPenalized(p)); -logger.debug("{} All peers appear to be penalized; returning null", this); -return null; +return retVal; } private List 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 4213dac..e1516d2 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,6 +27,8 @@ 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; @@ -40,6 +42,7 @@ import or
[nifi] branch master updated: NIFI-6569, NIFI-6570: Fixed bug that caused read timeouts not to occur with site-to-site. Fixed bug that caused site-to-site listener not to accept connections if there a
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 9ba2806 NIFI-6569, NIFI-6570: Fixed bug that caused read timeouts not to occur with site-to-site. Fixed bug that caused site-to-site listener not to accept connections if there are no input/output ports on the root group - this used to be a valid check and was done to prevent spawning extra threads and doing extra work if no ports exist but now that we have site-to-site ports outside of the root group it's no longer a reasonable condition to check. 9ba2806 is described below commit 9ba280680ff8b40b15d460e6f822c2b10d4373c3 Author: Mark Payne AuthorDate: Mon Aug 19 15:29:51 2019 -0400 NIFI-6569, NIFI-6570: Fixed bug that caused read timeouts not to occur with site-to-site. Fixed bug that caused site-to-site listener not to accept connections if there are no input/output ports on the root group - this used to be a valid check and was done to prevent spawning extra threads and doing extra work if no ports exist but now that we have site-to-site ports outside of the root group it's no longer a reasonable condition to check. This closes #3658. Signed-off-by: Koji Kawamura --- .../nifi/remote/io/socket/SocketChannelInput.java | 8 +- .../remote/io/socket/SocketChannelInputStream.java | 94 -- .../nifi/remote/SocketRemoteSiteListener.java | 20 + 3 files changed, 62 insertions(+), 60 deletions(-) diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java index 85ae504..5cf2a62 100644 --- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java +++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java @@ -16,13 +16,14 @@ */ package org.apache.nifi.remote.io.socket; +import org.apache.nifi.remote.io.InterruptableInputStream; +import org.apache.nifi.remote.protocol.CommunicationsInput; +import org.apache.nifi.stream.io.ByteCountingInputStream; + import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.channels.SocketChannel; -import org.apache.nifi.remote.io.InterruptableInputStream; -import org.apache.nifi.remote.protocol.CommunicationsInput; -import org.apache.nifi.stream.io.ByteCountingInputStream; public class SocketChannelInput implements CommunicationsInput { @@ -62,6 +63,7 @@ public class SocketChannelInput implements CommunicationsInput { public void interrupt() { interruptableIn.interrupt(); +socketIn.interrupt(); } @Override diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java index c0cfa11..21f1683 100644 --- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java +++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java @@ -16,19 +16,24 @@ */ package org.apache.nifi.remote.io.socket; +import org.apache.nifi.remote.exception.TransmissionDisabledException; + import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.net.SocketTimeoutException; import java.nio.ByteBuffer; +import java.nio.channels.SelectionKey; +import java.nio.channels.Selector; import java.nio.channels.SocketChannel; -import java.util.concurrent.TimeUnit; +import java.util.Set; public class SocketChannelInputStream extends InputStream { -private static final long CHANNEL_EMPTY_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(10, TimeUnit.MILLISECONDS); private final SocketChannel channel; private volatile int timeoutMillis = 3; +private volatile boolean interrupted = false; +private final Selector readSelector; private final ByteBuffer oneByteBuffer = ByteBuffer.allocate(1); private Byte bufferedByte = null; @@ -37,6 +42,9 @@ public class SocketChannelInputStream extends InputStream { // this class expects a non-blocking channel socketChannel.configureBlocking(false); this.channel = socketChannel; + +readSelector = Selector.open(); +this.channel.register(readSelector, SelectionKey.OP_READ); } public void setTimeout(final int timeoutMillis) { @@ -68,32 +76,24 @@ public class SocketChannelInputStream extends InputStream { final long maxTime = System.currentTimeMillis() + timeoutMillis; -final boolean blocking
[nifi] branch master updated: NIFI-6509 - Date related issue in unit test VolatileComponentStatusRepositoryTest
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 3de81d6 NIFI-6509 - Date related issue in unit test VolatileComponentStatusRepositoryTest 3de81d6 is described below commit 3de81d618ea45bd6cf26c33ccabf94b561733d67 Author: Tamas Palfy AuthorDate: Wed Aug 21 19:25:16 2019 +0200 NIFI-6509 - Date related issue in unit test VolatileComponentStatusRepositoryTest As NIFI-6508 has already solved the original issue, turned this PR into a refactor within in the affected Test class. This closes #3621. Signed-off-by: Koji Kawamura --- .../VolatileComponentStatusRepositoryTest.java | 147 +++-- 1 file changed, 74 insertions(+), 73 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java index dad852a..fa1baa1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java @@ -27,8 +27,8 @@ import org.junit.Test; import org.apache.nifi.util.NiFiProperties; import static junit.framework.TestCase.assertTrue; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyString; +import static org.apache.nifi.controller.status.history.VolatileComponentStatusRepository.DEFAULT_NUM_DATA_POINTS; +import static org.apache.nifi.controller.status.history.VolatileComponentStatusRepository.NUM_DATA_POINTS_PROPERTY; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.AssertJUnit.assertEquals; @@ -39,120 +39,121 @@ import static org.testng.AssertJUnit.assertEquals; */ public class VolatileComponentStatusRepositoryTest { - private static VolatileComponentStatusRepository repo1; - private static VolatileComponentStatusRepository repo2; - private static VolatileComponentStatusRepository repo3; + private static VolatileComponentStatusRepository filledRepo; + private static VolatileComponentStatusRepository partiallyFilledRepo; + private static VolatileComponentStatusRepository emptyRepo; private static final int FIVE_MINUTES = 30; private static int BUFSIZE3 = 10; @BeforeClass public static void createBuffers() { -NiFiProperties props1 = mock(NiFiProperties.class); -int BUFSIZE1 = 1_000_000; -when(props1.getIntegerProperty(anyString(), anyInt())).thenReturn(BUFSIZE1); -repo1 = new VolatileComponentStatusRepository(props1); // Fill the repo1 buffer completely with Date objects at five-minute intervals // This provides dates up to around Jul 1979 -for (long i = 0; i < BUFSIZE1; i++) { - repo1.timestamps.add(new Date(i * FIVE_MINUTES)); -} -assertEquals(BUFSIZE1, repo1.timestamps.getSize()); +filledRepo = initRepo(1_000_000, 0); -NiFiProperties props2 = mock(NiFiProperties.class); -int BUFSIZE2 = 1000; -when(props2.getIntegerProperty(anyString(), anyInt())).thenReturn(BUFSIZE2); -repo2 = new VolatileComponentStatusRepository(props2); -int OFFSET = 10; // Verify partially filled buffers work as expected. -for (long i = 0; i < BUFSIZE2 - OFFSET; i++) { - repo2.timestamps.add(new Date(i * FIVE_MINUTES)); +partiallyFilledRepo = initRepo(1000, 10); + +emptyRepo = createRepo(BUFSIZE3); + } + + private static VolatileComponentStatusRepository initRepo(int bufferSize, int offset) { +VolatileComponentStatusRepository repo = createRepo(bufferSize); + +for (long i = 0; i < bufferSize - offset; i++) { + repo.timestamps.add(new Date(i * FIVE_MINUTES)); } -assertEquals(BUFSIZE2 - OFFSET, repo2.timestamps.getSize()); +assertEquals(bufferSize - offset, repo.timestamps.getSize()); + +return repo; + } -NiFiProperties props3 = mock(NiFiProperties.class); -when(props3.getIntegerProperty(anyString(), anyInt())).thenReturn(BUFSIZE3); -repo3 = new VolatileComponentStatusRepository(props3); + private static VolatileComponentStatusRepository createRepo(int bufferSize) { +NiFiProperties props = mock(NiFiProperties.class); + +when(props.getIntegerProperty(NUM_DATA_POINTS_PROPERTY, DEFAULT_NUM_DATA_POINTS)).thenReturn(bufferSize); + +VolatileComponentStatusRepository repo = new VolatileComponentStatusR
[nifi] branch master updated: NIFI-6158 Fix conversion of Avro fixed type with logicalType decimal
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new cc88dd4 NIFI-6158 Fix conversion of Avro fixed type with logicalType decimal cc88dd4 is described below commit cc88dd428f6784e1bf8c72a52ed291fed96eb80b Author: Bryan Bende AuthorDate: Thu Aug 22 11:00:14 2019 -0400 NIFI-6158 Fix conversion of Avro fixed type with logicalType decimal This closes #3665. Signed-off-by: Koji Kawamura --- .../java/org/apache/nifi/avro/AvroTypeUtil.java| 5 + .../org/apache/nifi/avro/TestAvroTypeUtil.java | 34 +++ .../nifi-framework/nifi-properties-loader/pom.xml | 4 - .../nifi-parquet-processors/pom.xml| 7 ++ .../nifi/processors/parquet/FetchParquetTest.java | 108 +++-- .../resources/avro/user-with-fixed-decimal.avsc| 8 ++ pom.xml| 1 + 7 files changed, 131 insertions(+), 36 deletions(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java index a60c993..2bc95bc 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java @@ -1012,6 +1012,11 @@ public class AvroTypeUtil { return AvroTypeUtil.convertByteArray(bb.array()); case FIXED: final GenericFixed fixed = (GenericFixed) value; +final LogicalType fixedLogicalType = avroSchema.getLogicalType(); +if (fixedLogicalType != null && LOGICAL_TYPE_DECIMAL.equals(fixedLogicalType.getName())) { +final ByteBuffer fixedByteBuffer = ByteBuffer.wrap(fixed.bytes()); +return new Conversions.DecimalConversion().fromBytes(fixedByteBuffer, avroSchema, fixedLogicalType); +} return AvroTypeUtil.convertByteArray(fixed.bytes()); case ENUM: return value.toString(); 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 e60c164..0ecbe25 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 @@ -405,6 +405,40 @@ public class TestAvroTypeUtil { } @Test +public void testConvertAvroRecordToMapWithFieldTypeOfFixedAndLogicalTypeDecimal() { + // Create a field schema like {"type":"fixed","name":"amount","size":16,"logicalType":"decimal","precision":18,"scale":8} + final LogicalTypes.Decimal decimalType = LogicalTypes.decimal(18, 8); +final Schema fieldSchema = Schema.createFixed("amount", null, null, 16);; +decimalType.addToSchema(fieldSchema); + +// Create a field named "amount" using the field schema above +final Schema.Field field = new Schema.Field("amount", fieldSchema, null, (Object)null); + +// Create an overall record schema with the amount field +final Schema avroSchema = Schema.createRecord(Collections.singletonList(field)); + +// Create an example Avro record with the amount field of type fixed and a logical type of decimal +final BigDecimal expectedDecimalValue = new BigDecimal("1234567890.12345678"); +final GenericRecord genericRecord = new GenericData.Record(avroSchema); +genericRecord.put("amount", new Conversions.DecimalConversion().toFixed(expectedDecimalValue, fieldSchema, decimalType)); + +// Convert the Avro schema to a Record schema +final RecordSchema recordSchema = AvroTypeUtil.createSchema(avroSchema); + +// Convert the Avro record a Map and verify the object produced is the same BigDecimal that was converted to fixed +final Map convertedMap = AvroTypeUtil.convertAvroRecordToMap(genericRecord, recordSchema, StandardCharsets.UTF_8); +assertNotNull(convertedMap); +assertEquals(1, convertedMap.size()); + +final Object resultObj
[nifi] branch master updated: NIFI-6508 - Fix test failure caused by timezone and/or DST difference
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 00f0f84 NIFI-6508 - Fix test failure caused by timezone and/or DST difference 00f0f84 is described below commit 00f0f84856226ac39c92807c01945bb017e99bb4 Author: Malthe Borch AuthorDate: Wed Jul 31 11:09:44 2019 +0200 NIFI-6508 - Fix test failure caused by timezone and/or DST difference Signed-off-by: Koji Kawamura --- .../status/history/VolatileComponentStatusRepositoryTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java index e7532ff..dad852a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepositoryTest.java @@ -17,7 +17,7 @@ package org.apache.nifi.controller.status.history; import java.time.LocalDateTime; -import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.Date; import java.util.List; @@ -75,7 +75,7 @@ public class VolatileComponentStatusRepositoryTest { } private Date asDate(LocalDateTime localDateTime) { -return Date.from(localDateTime.atZone(ZoneId.systemDefault()).toInstant()); +return Date.from(localDateTime.toInstant(ZoneOffset.UTC)); } @Test
[nifi] branch master updated: NIFI-6477 Delete operate policy when its component is deleted
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 1cb1b00 NIFI-6477 Delete operate policy when its component is deleted 1cb1b00 is described below commit 1cb1b00c090afc85e5c4024a0339d5b508c231f8 Author: Koji Kawamura AuthorDate: Fri Aug 16 15:53:32 2019 +0900 NIFI-6477 Delete operate policy when its component is deleted +1 from markobean. This closes #3653. Signed-off-by: Koji Kawamura --- .../src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java | 1 + 1 file changed, 1 insertion(+) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 9f4744f..5ba1da6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -1729,6 +1729,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { resources.add(ResourceFactory.getProvenanceDataResource(componentResource)); resources.add(ResourceFactory.getDataTransferResource(componentResource)); resources.add(ResourceFactory.getPolicyResource(componentResource)); + resources.add(ResourceFactory.getOperationResource(componentResource)); for (final Resource resource : resources) { for (final RequestAction action : RequestAction.values()) {
[nifi] branch master updated: NIFI-6479 Fixes timezone issues in TestJdbcCommon Dates calculated for method testConvertToAvroStreamForDateTimeAsLogicalType now handle timezone correctly Added debug lo
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new b938fc2 NIFI-6479 Fixes timezone issues in TestJdbcCommon Dates calculated for method testConvertToAvroStreamForDateTimeAsLogicalType now handle timezone correctly Added debug logging of date/time calculations and expectations b938fc2 is described below commit b938fc20a3cf93547e3250f850db33eacb8ab083 Author: Jeff Storck AuthorDate: Wed Jul 24 14:14:25 2019 -0400 NIFI-6479 Fixes timezone issues in TestJdbcCommon Dates calculated for method testConvertToAvroStreamForDateTimeAsLogicalType now handle timezone correctly Added debug logging of date/time calculations and expectations This closes #3630. Signed-off-by: Koji Kawamura --- .../org/apache/nifi/util/db/TestJdbcCommon.java| 84 +++--- 1 file changed, 58 insertions(+), 26 deletions(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db/TestJdbcCommon.java b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db/TestJdbcCommon.java index 1f9d793..b58289b 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db/TestJdbcCommon.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db/TestJdbcCommon.java @@ -45,11 +45,18 @@ import java.sql.Statement; import java.sql.Time; import java.sql.Timestamp; import java.sql.Types; -import java.text.ParseException; -import java.text.SimpleDateFormat; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.time.temporal.TemporalAccessor; import java.util.HashSet; import java.util.Set; -import java.util.TimeZone; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; @@ -73,11 +80,13 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestJdbcCommon { +private static final Logger LOGGER = LoggerFactory.getLogger(TestJdbcCommon.class); static final String createTable = "create table restaurants(id integer, name varchar(20), city varchar(50))"; static final String dropTable = "drop table restaurants"; @@ -652,7 +661,7 @@ public class TestJdbcCommon { } @Test -public void testConvertToAvroStreamForDateTimeAsString() throws SQLException, IOException, ParseException { +public void testConvertToAvroStreamForDateTimeAsString() throws SQLException, IOException { final JdbcCommon.AvroConversionOptions options = JdbcCommon.AvroConversionOptions .builder().convertNames(true).useLogicalTypes(false).build(); @@ -664,7 +673,7 @@ public class TestJdbcCommon { } @Test -public void testConvertToAvroStreamForDateTimeAsLogicalType() throws SQLException, IOException, ParseException { +public void testConvertToAvroStreamForDateTimeAsLogicalType() throws SQLException, IOException { final JdbcCommon.AvroConversionOptions options = JdbcCommon.AvroConversionOptions .builder().convertNames(true).useLogicalTypes(true).build(); @@ -672,31 +681,56 @@ public class TestJdbcCommon { (record, date) -> { final int daysSinceEpoch = (int) record.get("date"); final long millisSinceEpoch = TimeUnit.MILLISECONDS.convert(daysSinceEpoch, TimeUnit.DAYS); -assertEquals(date, new java.sql.Date(millisSinceEpoch)); +java.sql.Date actual = java.sql.Date.valueOf(Instant.ofEpochMilli(millisSinceEpoch).atZone(ZoneOffset.UTC).toLocalDate()); +LOGGER.debug("comparing dates, expecting '{}', actual '{}'", date, actual); +assertEquals(date, actual); }, -(record, time) -> assertEquals(time, new Time((int) record.get("time"))), -(record, timestamp) -> assertEquals(timestamp, new Timestamp((long) record.get("timestamp"))) +(record, time) -> { +int millisSinceMidnight = (int) record.get("time"); +LocalTime localTime = Instant.ofEpochMilli(millisSinc
[nifi] branch master updated: NIFI-6407 - added support for useAvroLogicalTypes in PutBigQueryBatch
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 639e81e NIFI-6407 - added support for useAvroLogicalTypes in PutBigQueryBatch 639e81e is described below commit 639e81e5a12f5fccb35a581e009eee5bcdb4bff6 Author: Pierre Villard AuthorDate: Fri Jul 19 14:20:27 2019 +0200 NIFI-6407 - added support for useAvroLogicalTypes in PutBigQueryBatch fix Maven dep This closes #3592. Signed-off-by: Koji Kawamura --- .../nifi/processors/gcp/bigquery/BigQueryAttributes.java | 5 + .../apache/nifi/processors/gcp/bigquery/PutBigQueryBatch.java | 11 +++ .../nifi-gcp-bundle/nifi-gcp-services-api/pom.xml | 2 +- nifi-nar-bundles/nifi-gcp-bundle/pom.xml | 2 +- 4 files changed, 18 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/BigQueryAttributes.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/BigQueryAttributes.java index 842a176..81978eb 100644 --- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/BigQueryAttributes.java +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/BigQueryAttributes.java @@ -97,6 +97,11 @@ public class BigQueryAttributes { + "will skip when reading the data. The default value is 0. This property is useful if you have header rows in the " + "file that should be skipped."; +public static final String AVRO_USE_LOGICAL_TYPES_ATTR = "bq.avro.use.logical.types"; +public static final String AVRO_USE_LOGICAL_TYPES_DESC = "If format is set to Avro and if this option is set to true, you " ++ "can interpret logical types into their corresponding types (such as TIMESTAMP) instead of only using their raw " ++ "types (such as INTEGER)."; + // Batch Attributes diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatch.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatch.java index 5068ab5..5446c20 100644 --- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatch.java +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatch.java @@ -214,6 +214,15 @@ public class PutBigQueryBatch extends AbstractBigQueryProcessor { .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .build(); +public static final PropertyDescriptor AVRO_USE_LOGICAL_TYPES = new PropertyDescriptor.Builder() +.name(BigQueryAttributes.AVRO_USE_LOGICAL_TYPES_ATTR) +.displayName("Avro Input - Use Logical Types") +.description(BigQueryAttributes.AVRO_USE_LOGICAL_TYPES_DESC) +.required(true) +.allowableValues("true", "false") +.defaultValue("false") +.build(); + @Override public List getSupportedPropertyDescriptors() { return ImmutableList.builder() @@ -229,6 +238,7 @@ public class PutBigQueryBatch extends AbstractBigQueryProcessor { .add(CSV_FIELD_DELIMITER) .add(CSV_QUOTE) .add(CSV_SKIP_LEADING_ROWS) +.add(AVRO_USE_LOGICAL_TYPES) .build(); } @@ -280,6 +290,7 @@ public class PutBigQueryBatch extends AbstractBigQueryProcessor { .setCreateDisposition(JobInfo.CreateDisposition.valueOf(context.getProperty(CREATE_DISPOSITION).getValue())) .setWriteDisposition(JobInfo.WriteDisposition.valueOf(context.getProperty(WRITE_DISPOSITION).getValue())) .setIgnoreUnknownValues(context.getProperty(IGNORE_UNKNOWN).asBoolean()) + .setUseAvroLogicalTypes(context.getProperty(AVRO_USE_LOGICAL_TYPES).asBoolean()) .setMaxBadRecords(context.getProperty(MAXBAD_RECORDS).asInteger()) .setSchema(schema) .setFormatOptions(formatOption) diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml index 671474b..ea79c10 100644 --- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml @
[nifi] branch master updated: NIFI-6487 Add S3 User Metadata to ListS3 processor
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new cdee1d8 NIFI-6487 Add S3 User Metadata to ListS3 processor cdee1d8 is described below commit cdee1d8c09c52dd2aa78311cf8c8fe9b2f449d44 Author: JF Beauvais AuthorDate: Thu Jul 25 14:20:35 2019 +0200 NIFI-6487 Add S3 User Metadata to ListS3 processor Fix imports auto formatted by intellij NIFI-6487 Fix WriteAttribute documentation This closes #3603. Signed-off-by: Koji Kawamura --- .../org/apache/nifi/processors/aws/s3/ListS3.java | 46 + .../nifi/processors/aws/s3/AbstractS3IT.java | 9 .../apache/nifi/processors/aws/s3/ITListS3.java| 32 +++ .../apache/nifi/processors/aws/s3/TestListS3.java | 48 ++ 4 files changed, 117 insertions(+), 18 deletions(-) diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java index d3bade9..fb4e49f 100644 --- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java +++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java @@ -26,9 +26,19 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; import com.amazonaws.services.s3.model.GetObjectTaggingRequest; import com.amazonaws.services.s3.model.GetObjectTaggingResult; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.ListVersionsRequest; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import com.amazonaws.services.s3.model.S3VersionSummary; import com.amazonaws.services.s3.model.Tag; +import com.amazonaws.services.s3.model.VersionListing; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.PrimaryNodeOnly; @@ -53,14 +63,6 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.util.StandardValidators; import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListVersionsRequest; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.S3ObjectSummary; -import com.amazonaws.services.s3.model.S3VersionSummary; -import com.amazonaws.services.s3.model.VersionListing; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.ListObjectsV2Result; @PrimaryNodeOnly @TriggerSerially @@ -85,6 +87,8 @@ import com.amazonaws.services.s3.model.ListObjectsV2Result; @WritesAttribute(attribute = "s3.storeClass", description = "The storage class of the object"), @WritesAttribute(attribute = "s3.version", description = "The version of the object, if applicable"), @WritesAttribute(attribute = "s3.tag.___", description = "If 'Write Object Tags' is set to 'True', the tags associated to the S3 object that is being listed " + +"will be written as part of the flowfile attributes"), +@WritesAttribute(attribute = "s3.user.metadata.___", description = "If 'Write User Metadata' is set to 'True', the user defined metadata associated to the S3 object that is being listed " + "will be written as part of the flowfile attributes")}) @SeeAlso({FetchS3Object.class, PutS3Object.class, DeleteS3Object.class}) public class ListS3 extends AbstractS3Processor { @@ -150,8 +154,18 @@ public class ListS3 extends AbstractS3Processor { .defaultValue("false") .build(); +public static final PropertyDescriptor WRITE_USER_METADATA = new PropertyDescriptor.Builder() +.name("write-s3-user-metadata") +.displayName("Write User Metadata") +.description("If set to 'True', the user defined metadata associated with the S3 object will be written as FlowFile attributes") +.required(true) +.allowableValues(new AllowableValue("true", "True"), new AllowableValue("false", &qu
[nifi] branch master updated: NIFI-6490 MergeRecord supports Variable Registry for MIN_RECORDS and MAX_RECORDS
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 1d22e8a NIFI-6490 MergeRecord supports Variable Registry for MIN_RECORDS and MAX_RECORDS 1d22e8a is described below commit 1d22e8a86d001540bb86d017a9139393778628cb Author: Alessandro D'Armiento AuthorDate: Fri Jul 26 17:14:11 2019 +0200 NIFI-6490 MergeRecord supports Variable Registry for MIN_RECORDS and MAX_RECORDS Unified unit tests Added custom validation cases for MIN_RECORDS and MAX_RECORDS enforcing they are greater than zero. While MIN_RECORDS > 0 can fail individually, MAX_RECORDS > 0 validator cannot fail without having also at least another validation step (either the MIN_RECORDS > 0 or the MAX_RECORDS > MIN_RECORDS) to fail, since MIN_RECORDS is a required property with default value 1 This closes #3607. Signed-off-by: Koji Kawamura --- .../nifi/processors/standard/MergeRecord.java | 22 - .../standard/merge/RecordBinManager.java | 4 +- .../nifi/processors/standard/TestMergeRecord.java | 100 + 3 files changed, 122 insertions(+), 4 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java index 130d6b6..797359e 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeRecord.java @@ -179,6 +179,7 @@ public class MergeRecord extends AbstractSessionFactoryProcessor { .required(true) .defaultValue("1") .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) +.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .build(); public static final PropertyDescriptor MAX_RECORDS = new PropertyDescriptor.Builder() .name("max-records") @@ -188,6 +189,7 @@ public class MergeRecord extends AbstractSessionFactoryProcessor { .required(false) .defaultValue("1000") .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) +.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .build(); public static final PropertyDescriptor MAX_BIN_COUNT = new PropertyDescriptor.Builder() .name("max.bin.count") @@ -268,8 +270,8 @@ public class MergeRecord extends AbstractSessionFactoryProcessor { protected Collection customValidate(final ValidationContext validationContext) { final List results = new ArrayList<>(); -final Integer minRecords = validationContext.getProperty(MIN_RECORDS).asInteger(); -final Integer maxRecords = validationContext.getProperty(MAX_RECORDS).asInteger(); +final Integer minRecords = validationContext.getProperty(MIN_RECORDS).evaluateAttributeExpressions().asInteger(); +final Integer maxRecords = validationContext.getProperty(MAX_RECORDS).evaluateAttributeExpressions().asInteger(); if (minRecords != null && maxRecords != null && maxRecords < minRecords) { results.add(new ValidationResult.Builder() .subject("Max Records") @@ -278,6 +280,22 @@ public class MergeRecord extends AbstractSessionFactoryProcessor { .explanation(" property cannot be smaller than property") .build()); } +if (minRecords != null && minRecords <= 0) { +results.add(new ValidationResult.Builder() +.subject("Min Records") +.input(String.valueOf(minRecords)) +.valid(false) +.explanation(" property cannot be negative or zero") +.build()); +} +if (maxRecords != null && maxRecords <= 0) { +results.add(new ValidationResult.Builder() +.subject("Max Records") +.input(String.valueOf(maxRecords)) +.valid(false) +.explanation(" property cannot be negative or zero") +.build()); +} final Double minSize = validationContext.getProperty(MIN_SIZE).asDataSize(DataUnit.B); final Double maxSize = validationContext.getProperty(MAX_SIZE).asDataSize(DataUnit.B); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src
svn commit: r1863909 - in /nifi/site/trunk: mailing_lists.html people.html
Author: ijokarumawak Date: Mon Jul 29 03:13:13 2019 New Revision: 1863909 URL: http://svn.apache.org/viewvc?rev=1863909=rev Log: Deploy recent changes. Modified: nifi/site/trunk/mailing_lists.html nifi/site/trunk/people.html Modified: nifi/site/trunk/mailing_lists.html URL: http://svn.apache.org/viewvc/nifi/site/trunk/mailing_lists.html?rev=1863909=1863908=1863909=diff == --- nifi/site/trunk/mailing_lists.html (original) +++ nifi/site/trunk/mailing_lists.html Mon Jul 29 03:13:13 2019 @@ -230,9 +230,6 @@ IRC: #nifi on http://webchat.freenode.net/?channels=#nifi;>irc.freenode.net - -Apache Software Foundation HipChat: https://www.hipchat.com/gzh2m5YML;>#nifi - Modified: nifi/site/trunk/people.html URL: http://svn.apache.org/viewvc/nifi/site/trunk/people.html?rev=1863909=1863908=1863909=diff == --- nifi/site/trunk/people.html (original) +++ nifi/site/trunk/people.html Mon Jul 29 03:13:13 2019 @@ -363,6 +363,11 @@ Arpad Boda + +thenatog +Nathan Gough + +
[nifi-site] branch master updated: NIFI-6489 Remove reference to HipChat from mailing lists page (#35)
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi-site.git The following commit(s) were added to refs/heads/master by this push: new 62f4219 NIFI-6489 Remove reference to HipChat from mailing lists page (#35) 62f4219 is described below commit 62f4219860c1ff40c4e3b07448bd42a830fb4ec4 Author: Andrew Lim AuthorDate: Sun Jul 28 22:48:38 2019 -0400 NIFI-6489 Remove reference to HipChat from mailing lists page (#35) --- src/pages/html/mailing_lists.hbs | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/pages/html/mailing_lists.hbs b/src/pages/html/mailing_lists.hbs index 1a5da3c..7ebf36c 100644 --- a/src/pages/html/mailing_lists.hbs +++ b/src/pages/html/mailing_lists.hbs @@ -126,9 +126,6 @@ title: Apache NiFi Mailing Lists Chat IRC: #nifi on http://webchat.freenode.net/?channels=#nifi;>irc.freenode.net - -Apache Software Foundation HipChat: https://www.hipchat.com/gzh2m5YML;>#nifi -
[nifi] branch master updated: NIFI-6460 Remove references to HipChat, add links to Apache NiFi Slack
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 66e3617 NIFI-6460 Remove references to HipChat, add links to Apache NiFi Slack 66e3617 is described below commit 66e3617ba73808515a04f85bda4130e9a1ff2efd Author: Andrew Lim AuthorDate: Thu Jul 25 16:18:10 2019 -0400 NIFI-6460 Remove references to HipChat, add links to Apache NiFi Slack This closes #3605. Signed-off-by: Koji Kawamura --- README.md | 10 +++--- nifi-assembly/README.md | 30 +- 2 files changed, 24 insertions(+), 16 deletions(-) diff --git a/README.md b/README.md index ec339ac..3e41438 100644 --- a/README.md +++ b/README.md @@ -126,9 +126,13 @@ To run NiFi: ## Getting Help If you have questions, you can reach out to our mailing list: d...@nifi.apache.org -([archive](http://mail-archives.apache.org/mod_mbox/nifi-dev)). -For more interactive conversations and chat, we're also often available in IRC: #nifi on -[irc.freenode.net](http://webchat.freenode.net/?channels=#nifi) and in #NiFi on [ASF HipChat](https://www.hipchat.com/gzh2m5YML). +([archive](http://mail-archives.apache.org/mod_mbox/nifi-dev)). For more interactive discussions, community members can often be found in the following locations: + +- Apache NiFi Slack Workspace: https://apachenifi.slack.com/ + + New users can join the workspace using the following [invite link](https://join.slack.com/t/apachenifi/shared_invite/enQtNDI2NDMyMTY3MTA5LWJmZDI3MmM1ZmYyODQwZDYwM2MyMDY5ZjkyMDkxY2JmOGMyNmEzYTE0MTRkZTYwYzZlYTJkY2JhZTYyMzcyZGI). + +- IRC: #nifi on [irc.freenode.net](http://webchat.freenode.net/?channels=#nifi) ## Documentation diff --git a/nifi-assembly/README.md b/nifi-assembly/README.md index c39f51f..5095b46 100644 --- a/nifi-assembly/README.md +++ b/nifi-assembly/README.md @@ -55,9 +55,13 @@ To start NiFi: ## Getting Help If you have questions, you can reach out to our mailing list: d...@nifi.apache.org -([archive](http://mail-archives.apache.org/mod_mbox/nifi-dev)). -We're also often available in IRC: #nifi on -[irc.freenode.net](http://webchat.freenode.net/?channels=#nifi). +([archive](http://mail-archives.apache.org/mod_mbox/nifi-dev)). For more interactive discussions, community members can often be found in the following locations: + +- Apache NiFi Slack Workspace: https://apachenifi.slack.com/ + + New users can join the workspace using the following [invite link](https://join.slack.com/t/apachenifi/shared_invite/enQtNDI2NDMyMTY3MTA5LWJmZDI3MmM1ZmYyODQwZDYwM2MyMDY5ZjkyMDkxY2JmOGMyNmEzYTE0MTRkZTYwYzZlYTJkY2JhZTYyMzcyZGI). + +- IRC: #nifi on [irc.freenode.net](http://webchat.freenode.net/?channels=#nifi) ## Requirements * JDK 1.8 or higher @@ -81,28 +85,28 @@ limitations under the License. ## Export Control -This distribution includes cryptographic software. The country in which you +This distribution includes cryptographic software. The country in which you currently reside may have restrictions on the import, possession, use, and/or -re-export to another country, of encryption software. BEFORE using any -encryption software, please check your country's laws, regulations and +re-export to another country, of encryption software. BEFORE using any +encryption software, please check your country's laws, regulations and policies concerning the import, possession, or use, and re-export of encryption software, to see if this is permitted. See <http://www.wassenaar.org/> for more information. -The U.S. Government Department of Commerce, Bureau of Industry and Security -(BIS), has classified this software as Export Commodity Control Number (ECCN) -5D002.C.1, which includes information security software using or performing +The U.S. Government Department of Commerce, Bureau of Industry and Security +(BIS), has classified this software as Export Commodity Control Number (ECCN) +5D002.C.1, which includes information security software using or performing cryptographic functions with asymmetric algorithms. The form and manner of this -Apache Software Foundation distribution makes it eligible for export under the +Apache Software Foundation distribution makes it eligible for export under the License Exception ENC Technology Software Unrestricted (TSU) exception (see the BIS Export Administration Regulations, Section 740.13) for both object code and source code. -The following provides more details on the included cryptographic software: +The following provides more details on the included cryptographic software: -Apache NiFi uses BouncyCastle, Jasypt, JCraft Inc., and the built-in +Apache NiFi uses BouncyCastle, Jasypt, JCraft Inc., and the built-in java cryptography libraries for SSL, SSH, and the protection -of sensitive configuration parameter
[nifi] branch master updated: NIFI-6442: ExecuteSQL/ExecuteSQLRecord convert to Avro date type incorrectly when set 'Use Avro Logical Types' to true
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 3bfef33 NIFI-6442: ExecuteSQL/ExecuteSQLRecord convert to Avro date type incorrectly when set 'Use Avro Logical Types' to true 3bfef33 is described below commit 3bfef3356e158d8691008e3675bb7a5cc65064fb Author: archon AuthorDate: Wed Jul 17 10:12:24 2019 +0800 NIFI-6442: ExecuteSQL/ExecuteSQLRecord convert to Avro date type incorrectly when set 'Use Avro Logical Types' to true This closes #3584. Signed-off-by: Koji Kawamura --- .../nifi/serialization/record/util/DataTypeUtils.java | 16 .../main/java/org/apache/nifi/avro/AvroTypeUtil.java| 8 +++- .../java/org/apache/nifi/avro/TestAvroTypeUtil.java | 17 + 3 files changed, 28 insertions(+), 13 deletions(-) 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 bab455a..0686dcf 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 @@ -892,15 +892,15 @@ public class DataTypeUtils { return null; } +if (value instanceof Date) { +return (Date) value; +} + if (value instanceof java.util.Date) { java.util.Date _temp = (java.util.Date)value; return new Date(_temp.getTime()); } -if (value instanceof Date) { -return (Date) value; -} - if (value instanceof Number) { final long longValue = ((Number) value).longValue(); return new Date(longValue); @@ -1038,14 +1038,14 @@ public class DataTypeUtils { return null; } -if (value instanceof java.util.Date) { -return new Timestamp(((java.util.Date)value).getTime()); -} - if (value instanceof Timestamp) { return (Timestamp) value; } +if (value instanceof java.util.Date) { +return new Timestamp(((java.util.Date)value).getTime()); +} + if (value instanceof Number) { final long longValue = ((Number) value).longValue(); return new Timestamp(longValue); diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java index 097844a..a60c993 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java @@ -60,6 +60,7 @@ import java.sql.Blob; import java.sql.Time; import java.sql.Timestamp; import java.time.Duration; +import java.time.LocalDate; import java.time.temporal.ChronoUnit; import java.util.AbstractMap; import java.util.ArrayList; @@ -652,9 +653,8 @@ public class AvroTypeUtil { if (LOGICAL_TYPE_DATE.equals(logicalType.getName())) { final String format = AvroTypeUtil.determineDataType(fieldSchema).getFormat(); -final Date date = DataTypeUtils.toDate(rawValue, () -> DataTypeUtils.getDateFormat(format), fieldName); -final Duration duration = Duration.between(new Date(0L).toInstant(), new Date(date.getTime()).toInstant()); -final long days = duration.toDays(); +final java.sql.Date date = DataTypeUtils.toDate(rawValue, () -> DataTypeUtils.getDateFormat(format), fieldName); +final long days = ChronoUnit.DAYS.between(LocalDate.ofEpochDay(0), date.toLocalDate()); return (int) days; } else if (LOGICAL_TYPE_TIME_MILLIS.equals(logicalType.getName())) { final String format = AvroTypeUtil.determineDataType(fieldSchema).getFormat(); @@ -679,8 +679,6 @@ public class AvroTypeUtil { final Duration duration = Duration.between(date.toInstant().truncatedTo(ChronoUnit.DAYS), date.toInstant()); return duration.toMillis() * 1000L; } else if (LOGICAL_TYPE_TIMESTAMP_MILLIS.equals(logicalType.getName())) { -final String format = AvroTypeUtil.determineDataType(fieldSchema).getFormat(); -Timestamp t = DataTypeUtils.toTimestamp(ra
[nifi] branch master updated: NIFI-6334 - fix custom validate error in PutBigqueryBatch
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 630c651 NIFI-6334 - fix custom validate error in PutBigqueryBatch 630c651 is described below commit 630c651226228ab2ed990d679b7588b93303682e Author: Pierre Villard AuthorDate: Thu Jul 18 15:48:39 2019 +0200 NIFI-6334 - fix custom validate error in PutBigqueryBatch This closes #3589. Signed-off-by: Koji Kawamura --- .../processors/gcp/bigquery/AbstractBigQueryProcessor.java| 3 ++- .../nifi/processors/gcp/bigquery/PutBigQueryBatchTest.java| 11 ++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/AbstractBigQueryProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/AbstractBigQueryProcessor.java index c249e7e..3751060 100644 --- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/AbstractBigQueryProcessor.java +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/AbstractBigQueryProcessor.java @@ -35,6 +35,7 @@ import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory; import org.apache.nifi.proxy.ProxyConfiguration; import org.apache.nifi.util.StringUtils; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -135,7 +136,7 @@ public abstract class AbstractBigQueryProcessor extends AbstractGCPProcessor customValidate(ValidationContext validationContext) { -final Collection results = super.customValidate(validationContext); +final Collection results = new ArrayList(super.customValidate(validationContext)); ProxyConfiguration.validateProxySpec(validationContext, results, ProxyAwareTransportFactory.PROXY_SPECS); final boolean projectId = validationContext.getProperty(PROJECT_ID).isSet(); diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatchTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatchTest.java index 7ec5aa9..c4063b3 100644 --- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatchTest.java +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/bigquery/PutBigQueryBatchTest.java @@ -125,7 +125,6 @@ public class PutBigQueryBatchTest extends AbstractBQTest { runner.assertAllFlowFilesTransferred(PutBigQueryBatch.REL_SUCCESS); } - @Test public void testFailedLoad() throws Exception { when(table.exists()).thenReturn(Boolean.TRUE); @@ -150,4 +149,14 @@ public class PutBigQueryBatchTest extends AbstractBQTest { runner.assertAllFlowFilesTransferred(PutBigQueryBatch.REL_FAILURE); } + +@Test +public void testMandatoryProjectId() throws Exception { +final TestRunner runner = buildNewRunner(getProcessor()); +addRequiredPropertiesToRunner(runner); +runner.assertValid(); + +runner.removeProperty(PutBigQueryBatch.PROJECT_ID); +runner.assertNotValid(); +} } \ No newline at end of file
[nifi] branch master updated: NIFI-6432 Updating get, containsKey, and remove in HBase_1_1_2_ClientMapCacheService and HBase_2_ClientMapCacheService so that they use the column family and qualifier sp
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new f2db153 NIFI-6432 Updating get, containsKey, and remove in HBase_1_1_2_ClientMapCacheService and HBase_2_ClientMapCacheService so that they use the column family and qualifier specified in the service f2db153 is described below commit f2db1539a84c4e265edc9ad0c403fe61e7c57c6c Author: Bryan Bende AuthorDate: Thu Jul 11 14:31:13 2019 -0400 NIFI-6432 Updating get, containsKey, and remove in HBase_1_1_2_ClientMapCacheService and HBase_2_ClientMapCacheService so that they use the column family and qualifier specified in the service This closes #3581. Signed-off-by: Koji Kawamura --- .../hbase/HBase_1_1_2_ClientMapCacheService.java | 32 +++--- .../nifi/hbase/HBase_2_ClientMapCacheService.java | 32 +++--- 2 files changed, 32 insertions(+), 32 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java index 0690db6..d2dc155 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java @@ -16,12 +16,6 @@ */ package org.apache.nifi.hbase; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; @@ -29,22 +23,25 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; - import org.apache.nifi.distributed.cache.client.AtomicCacheEntry; import org.apache.nifi.distributed.cache.client.AtomicDistributedMapCacheClient; +import org.apache.nifi.distributed.cache.client.Deserializer; import org.apache.nifi.distributed.cache.client.Serializer; import org.apache.nifi.expression.ExpressionLanguageScope; -import org.apache.nifi.distributed.cache.client.Deserializer; -import org.apache.nifi.reporting.InitializationException; - -import java.nio.charset.StandardCharsets; +import org.apache.nifi.hbase.put.PutColumn; +import org.apache.nifi.hbase.scan.Column; import org.apache.nifi.hbase.scan.ResultCell; import org.apache.nifi.hbase.scan.ResultHandler; -import org.apache.nifi.hbase.scan.Column; -import org.apache.nifi.hbase.put.PutColumn; - - import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.reporting.InitializationException; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import static org.apache.nifi.hbase.VisibilityLabelUtils.AUTHORIZATIONS; @@ -164,6 +161,7 @@ public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService final HBaseRowHandler handler = new HBaseRowHandler(); final List columnsList = new ArrayList(0); + columnsList.add(new Column(hBaseColumnFamilyBytes, hBaseColumnQualifierBytes)); hBaseClientService.scan(hBaseCacheTableName, rowIdBytes, rowIdBytes, columnsList, authorizations, handler); return (handler.numRows() > 0); @@ -196,6 +194,7 @@ public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService final HBaseRowHandler handler = new HBaseRowHandler(); final List columnsList = new ArrayList(0); + columnsList.add(new Column(hBaseColumnFamilyBytes, hBaseColumnQualifierBytes)); hBaseClientService.scan(hBaseCacheTableName, rowIdBytes, rowIdBytes, columnsList, authorizations, handler); if (handler.numRows() > 1) { @@ -212,7 +211,8 @@ public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService final boolean contains = containsKey(key, keySerializer); if (contains) { final byte[] rowIdBytes = serialize(key, keySerializer); -hBaseClientService.delete(hBaseCacheTableName, rowI
[nifi] 01/02: NIFI-6419: Fixed AvroWriter single record with external schema results in data loss
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit 24e50953a3e2dde4cda0261e4f7dba083d5ed602 Author: Peter Turcsanyi AuthorDate: Thu Jul 4 18:48:36 2019 +0200 NIFI-6419: Fixed AvroWriter single record with external schema results in data loss This closes #3573. Signed-off-by: Koji Kawamura --- .../nifi-record-serialization-services/pom.xml | 1 + .../org/apache/nifi/avro/TestWriteAvroResult.java | 69 ++ .../nifi/avro/TestWriteAvroResultWithSchema.java | 18 +- .../avro/TestWriteAvroResultWithoutSchema.java | 15 - .../src/test/resources/avro/simple.avsc| 8 +++ 5 files changed, 109 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml index 06fdb6d..e32ec28 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml @@ -136,6 +136,7 @@ src/test/resources/avro/logical-types.avsc src/test/resources/avro/logical-types-nullable.avsc src/test/resources/avro/multiple-types.avsc +src/test/resources/avro/simple.avsc src/test/resources/csv/extra-white-space.csv src/test/resources/csv/multi-bank-account.csv src/test/resources/csv/single-bank-account.csv 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 4751f74..d3e5f6c 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 @@ -26,6 +26,7 @@ import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.WriteResult; import org.apache.nifi.serialization.record.DataType; +import org.apache.nifi.serialization.record.ListRecordSet; import org.apache.nifi.serialization.record.MapRecord; import org.apache.nifi.serialization.record.Record; import org.apache.nifi.serialization.record.RecordField; @@ -66,10 +67,78 @@ public abstract class TestWriteAvroResult { protected abstract GenericRecord readRecord(InputStream in, Schema schema) throws IOException; +protected abstract List readRecords(InputStream in, Schema schema, int recordCount) throws IOException; + protected void verify(final WriteResult writeResult) { } @Test +public void testWriteRecord() throws IOException { +final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/simple.avsc")); + +final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + +final List fields = new ArrayList<>(); +fields.add(new RecordField("msg", RecordFieldType.STRING.getDataType())); +final RecordSchema recordSchema = new SimpleRecordSchema(fields); + +final Map values = new HashMap<>(); +values.put("msg", "nifi"); +final Record record = new MapRecord(recordSchema, values); + +try (final RecordSetWriter writer = createWriter(schema, baos)) { +writer.write(record); +} + +final byte[] data = baos.toByteArray(); + +try (final InputStream in = new ByteArrayInputStream(data)) { +final GenericRecord avroRecord = readRecord(in, schema); + +assertNotNull(avroRecord); +assertNotNull(avroRecord.get("msg")); +assertEquals("nifi", avroRecord.get("msg").toString()); +} +} + +@Test +public void testWriteRecordSet() throws IOException { +final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/simple.avsc")); + +final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + +final List fields = new A
[nifi] branch master updated (4783b12 -> e277545)
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from 4783b12 NIFI-5254 Upgraded Groovy to version 2.5.4 Updated pom files regarding groovy dependencies. groovy-all was changed to a pom type dependency Added Groovy Bintray as a plugin repository in root pom Upgraded spock-core to version 1.3-groovy-2.5 and added exclusion of groovy dependencies to force spock to use the version from dependencyManagement Updated groovy-eclipse-batch to use the groovy.version property to determine the version Updated groovy-eclipse-compiler to 3.4.0 [...] new 24e5095 NIFI-6419: Fixed AvroWriter single record with external schema results in data loss new e277545 NIFI-6419: Flush the buffers on close to ensure all data is written. The 2 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: .../nifi-record-serialization-services/pom.xml | 1 + .../avro/WriteAvroResultWithExternalSchema.java| 1 + .../org/apache/nifi/avro/TestWriteAvroResult.java | 69 ++ .../nifi/avro/TestWriteAvroResultWithSchema.java | 18 +- .../avro/TestWriteAvroResultWithoutSchema.java | 15 - .../src/test/resources/avro/simple.avsc| 8 +++ 6 files changed, 110 insertions(+), 2 deletions(-) create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/avro/simple.avsc
[nifi] branch master updated: NIFI-6271, fix issue that incoming flowfile attributes don't copy into output flowfiles when Output Batch Size is set
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new fa1ed16 NIFI-6271, fix issue that incoming flowfile attributes don't copy into output flowfiles when Output Batch Size is set fa1ed16 is described below commit fa1ed16e2bfc389466941e3bfe36a7a6ff08403b Author: avseq1234 AuthorDate: Sun Jul 7 21:26:16 2019 +0800 NIFI-6271, fix issue that incoming flowfile attributes don't copy into output flowfiles when Output Batch Size is set NIFI-6271, fix incoming flowfile attributes don't copy into output flowfiles when Output Batch Size is set NIFI-6271, fix incoming flowfile attributes don't copy into output flowfiles when Output Batch Size is set replace getAttribute(uuid) with getAttribute(CoreAttributes.UUID.key() fix checkstyle violation This closes #3575. Signed-off-by: Koji Kawamura --- .../nifi/processors/standard/AbstractExecuteSQL.java | 12 +++- .../apache/nifi/processors/standard/TestExecuteSQL.java| 14 -- .../nifi/processors/standard/TestExecuteSQLRecord.java | 9 - 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java index 212febc..700e92e 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java @@ -62,6 +62,7 @@ public abstract class AbstractExecuteSQL extends AbstractProcessor { public static final String RESULT_QUERY_FETCH_TIME = "executesql.query.fetchtime"; public static final String RESULTSET_INDEX = "executesql.resultset.index"; public static final String RESULT_ERROR_MESSAGE = "executesql.error.message"; +public static final String INPUT_FLOWFILE_UUID = "input.flowfile.uuid"; public static final String FRAGMENT_ID = FragmentAttributes.FRAGMENT_ID.key(); public static final String FRAGMENT_INDEX = FragmentAttributes.FRAGMENT_INDEX.key(); @@ -247,6 +248,8 @@ public abstract class AbstractExecuteSQL extends AbstractProcessor { boolean hasUpdateCount = st.getUpdateCount() != -1; +Map inputFileAttrMap = fileToProcess == null ? null : fileToProcess.getAttributes(); +String inputFileUUID = fileToProcess == null ? null : fileToProcess.getAttribute(CoreAttributes.UUID.key()); while (hasResults || hasUpdateCount) { //getMoreResults() and execute() return false to indicate that the result of the statement is just a number and not a ResultSet if (hasResults) { @@ -262,9 +265,13 @@ public abstract class AbstractExecuteSQL extends AbstractProcessor { resultSetFF = session.create(); } else { resultSetFF = session.create(fileToProcess); -resultSetFF = session.putAllAttributes(resultSetFF, fileToProcess.getAttributes()); } +if (inputFileAttrMap != null) { +resultSetFF = session.putAllAttributes(resultSetFF, inputFileAttrMap); +} + + try { resultSetFF = session.write(resultSetFF, out -> { try { @@ -283,6 +290,9 @@ public abstract class AbstractExecuteSQL extends AbstractProcessor { attributesToAdd.put(RESULT_QUERY_EXECUTION_TIME, String.valueOf(executionTimeElapsed)); attributesToAdd.put(RESULT_QUERY_FETCH_TIME, String.valueOf(fetchTimeElapsed)); attributesToAdd.put(RESULTSET_INDEX, String.valueOf(resultCount)); +if (inputFileUUID != null) { +attributesToAdd.put(INPUT_FLOWFILE_UUID, inputFileUUID); +} attributesToAdd.putAll(sqlWriter.getAttributesToAdd()); resultSetFF = session.putAllAttributes(resultSetFF, attributesToAdd); sqlWriter.updateCounters(session); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processor
[nifi] branch master updated: NIFI-6174: Add ClientAuth property to ListenBeats
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 4166392 NIFI-6174: Add ClientAuth property to ListenBeats 4166392 is described below commit 41663929a4727592972e6be04b3c516a752e760e Author: DimDroll AuthorDate: Wed Jun 19 15:59:52 2019 +0300 NIFI-6174: Add ClientAuth property to ListenBeats This closes #3539. Signed-off-by: Koji Kawamura --- .../apache/nifi/processors/beats/ListenBeats.java | 30 +++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java index af5d1b4..6a4a568 100644 --- a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java +++ b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java @@ -28,6 +28,8 @@ import java.util.Map; import java.util.concurrent.BlockingQueue; import javax.net.ssl.SSLContext; +import org.apache.nifi.security.util.SslContextFactory; +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.WritesAttribute; @@ -85,11 +87,21 @@ public class ListenBeats extends AbstractListenEventBatchingProcessor getAdditionalProperties() { return Arrays.asList( MAX_CONNECTIONS, -SSL_CONTEXT_SERVICE +SSL_CONTEXT_SERVICE, +CLIENT_AUTH ); } @@ -105,6 +117,14 @@ public class ListenBeats extends AbstractListenEventBatchingProcessor(eventFactory, handlerFactory, bufferPool, events, -getLogger(), maxConnections, sslContext, charSet); +getLogger(), maxConnections, sslContext, clientAuth, charSet); }
[nifi] branch master updated: NIFI-6004 PutFile created directory permissions
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new f1ae059 NIFI-6004 PutFile created directory permissions f1ae059 is described below commit f1ae05974e305b8487b26a87338d2faf9c2d851f Author: adyoun2 AuthorDate: Thu Feb 7 16:41:48 2019 + NIFI-6004 PutFile created directory permissions NIFI-6004 Improve testing of PutFile file and directory permissions NIFI-6004 Typo in regex NIFI-6004 Updates based on review This closes #3294. Signed-off-by: Koji Kawamura --- .../apache/nifi/processors/standard/PutFile.java | 128 ++--- .../nifi/processors/standard/TestPutFile.java | 116 +-- 2 files changed, 220 insertions(+), 24 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java index 8784c75..73caa69 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java @@ -28,6 +28,9 @@ import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.RequiredPermission; +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.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -58,6 +61,7 @@ import java.util.List; import java.util.Locale; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; import java.util.regex.Pattern; @EventDriven @@ -83,6 +87,28 @@ public class PutFile extends AbstractProcessor { public static final String FILE_MODIFY_DATE_ATTRIBUTE = "file.lastModifiedTime"; public static final String FILE_MODIFY_DATE_ATTR_FORMAT = "-MM-dd'T'HH:mm:ssZ"; +public static final Pattern RWX_PATTERN = Pattern.compile("^([r-][w-])([x-])([r-][w-])([x-])([r-][w-])([x-])$"); +public static final Pattern NUM_PATTERN = Pattern.compile("^[0-7]{3}$"); + +private static final Validator PERMISSIONS_VALIDATOR = new Validator() { +@Override +public ValidationResult validate(String subject, String input, ValidationContext context) { +ValidationResult.Builder vr = new ValidationResult.Builder(); +if (context.isExpressionLanguagePresent(input)) { +return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); +} + +if (RWX_PATTERN.matcher(input).matches() || NUM_PATTERN.matcher(input).matches()) { +return vr.valid(true).build(); +} +return vr.valid(false) +.subject(subject) +.input(input) +.explanation("This must be expressed in rwxr-x--- form or octal triplet form.") +.build(); +} +}; + public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder() .name("Directory") .description("The directory to which files should be written. You may use expression language such as /aa/bb/${path}") @@ -117,13 +143,13 @@ public class PutFile extends AbstractProcessor { + "place of denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). You may also use expression language such as " + "${file.permissions}.") .required(false) -.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.addValidator(PERMISSIONS_VALIDATOR) .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .build(); public static final PropertyDescriptor CHANGE_OWNER = new PropertyDescriptor.Builder() .name("Owner") .description("Sets the owner on the output file to the value of this attribute. You may also use expression language such as " -+ "${file.owner}.") ++ "${file.owner}. Note on many operating systems Nifi must be running as a super-user to have t
[nifi] branch master updated: NIFI-6362 - Upgraded com.puppycrawl.tools:checkstyle to 8.18. Made small checkstyle changes to nifi-kafka.
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 0e1a37f NIFI-6362 - Upgraded com.puppycrawl.tools:checkstyle to 8.18. Made small checkstyle changes to nifi-kafka. 0e1a37f is described below commit 0e1a37fcb9bc6fcc5c801a34d19673745c908437 Author: thenatog AuthorDate: Thu Jun 6 18:15:49 2019 -0400 NIFI-6362 - Upgraded com.puppycrawl.tools:checkstyle to 8.18. Made small checkstyle changes to nifi-kafka. NIFI-6362 - Upgraded both checkstyle plugins to latest. Added a checkstyleCache file and property. NIFI-6362 - Downgraded maven-checkstyle-plugin to 3.0.0 instead as there was an issue with the latest version and @throws Javadoc checkstyle module. Fixed some checkstyle issues. Changed checkstyleCache file to .checkstyleCache. Added to .gitignore. NIFI-6362 - Changed checkstyleCache file location into target/, and is no longer a hidden file. Removed .checkstyleCache from .gitignore as it is no longer required. NIFI-6362 - Grep pattern no longer required. This closes #3522. Signed-off-by: Koji Kawamura --- .../main/java/org/apache/nifi/atlas/security/AtlasAuthN.java | 2 +- .../coordination/http/replication/RequestReplicator.java | 2 ++ .../java/org/apache/nifi/remote/protocol/ServerProtocol.java | 10 +- .../org/apache/nifi/web/api/concurrent/RequestManager.java | 1 + .../nifi/processors/kafka/pubsub/KafkaProcessorUtils.java | 2 +- .../nifi/processors/kafka/pubsub/KafkaProcessorUtils.java | 2 +- .../nifi/processors/kafka/pubsub/KafkaProcessorUtils.java | 2 +- .../nifi/processors/kafka/pubsub/KafkaProcessorUtils.java | 2 +- .../nifi/processors/kafka/pubsub/KafkaProcessorUtils.java | 2 +- pom.xml| 8 +--- 10 files changed, 19 insertions(+), 14 deletions(-) diff --git a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/security/AtlasAuthN.java b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/security/AtlasAuthN.java index a0b036e..660c09b 100644 --- a/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/security/AtlasAuthN.java +++ b/nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/security/AtlasAuthN.java @@ -35,7 +35,7 @@ public interface AtlasAuthN { * Populate required Atlas application properties. * This method is called when Atlas reporting task generates atlas-application.properties. */ -default void populateProperties(final Properties properties){}; +default void populateProperties(final Properties properties){} default Optional validateRequiredField(ValidationContext context, PropertyDescriptor prop) { if (!context.getProperty(prop).isSet()) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java index 8a98ed7..6af6111 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java @@ -22,6 +22,8 @@ import java.util.Map; import java.util.Set; import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException; +import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException; import org.apache.nifi.cluster.protocol.NodeIdentifier; public interface RequestReplicator { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java index dede80f..e5cece6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java @@ -74,7 +74,7 @@ public interface ServerProtocol extends VersionedRemoteResource { * @param
[nifi] branch master updated: NIFI-6243 Add Support for AtomicDistributedCache to the HBase 1.x and 2.x Map Cache Services
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new e913f57 NIFI-6243 Add Support for AtomicDistributedCache to the HBase 1.x and 2.x Map Cache Services e913f57 is described below commit e913f5706f3b3a0a2e98cf77ac4c96f63ba4f104 Author: Shawn Weeks AuthorDate: Sat May 4 08:58:00 2019 -0500 NIFI-6243 Add Support for AtomicDistributedCache to the HBase 1.x and 2.x Map Cache Services This closes #3462. Signed-off-by: Koji Kawamura --- .../hbase/HBase_1_1_2_ClientMapCacheService.java | 35 +- .../apache/nifi/hbase/MockHBaseClientService.java | 14 ++-- .../TestHBase_1_1_2_ClientMapCacheService.java | 77 +- .../nifi/hbase/HBase_2_ClientMapCacheService.java | 35 +- .../apache/nifi/hbase/MockHBaseClientService.java | 14 ++-- .../hbase/TestHBase_2_ClientMapCacheService.java | 77 +- 6 files changed, 232 insertions(+), 20 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java old mode 100644 new mode 100755 index 3991584..0690db6 --- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientMapCacheService.java @@ -30,7 +30,8 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; -import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; +import org.apache.nifi.distributed.cache.client.AtomicCacheEntry; +import org.apache.nifi.distributed.cache.client.AtomicDistributedMapCacheClient; import org.apache.nifi.distributed.cache.client.Serializer; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.distributed.cache.client.Deserializer; @@ -52,7 +53,7 @@ import static org.apache.nifi.hbase.VisibilityLabelUtils.AUTHORIZATIONS; @CapabilityDescription("Provides the ability to use an HBase table as a cache, in place of a DistributedMapCache." + " Uses a HBase_1_1_2_ClientService controller to communicate with HBase.") -public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService implements DistributedMapCacheClient { +public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService implements AtomicDistributedMapCacheClient { static final PropertyDescriptor HBASE_CLIENT_SERVICE = new PropertyDescriptor.Builder() .name("HBase Client Service") @@ -229,6 +230,36 @@ public class HBase_1_1_2_ClientMapCacheService extends AbstractControllerService protected void finalize() throws Throwable { } +@Override +public AtomicCacheEntry fetch(K key, Serializer keySerializer, Deserializer valueDeserializer) throws IOException { +final byte[] rowIdBytes = serialize(key, keySerializer); +final HBaseRowHandler handler = new HBaseRowHandler(); + +final List columnsList = new ArrayList<>(1); +columnsList.add(new Column(hBaseColumnFamilyBytes, hBaseColumnQualifierBytes)); + +hBaseClientService.scan(hBaseCacheTableName, rowIdBytes, rowIdBytes, columnsList, authorizations, handler); + +if (handler.numRows() > 1) { +throw new IOException("Found multiple rows in HBase for key"); +} else if (handler.numRows() == 1) { +return new AtomicCacheEntry<>(key, deserialize(handler.getLastResultBytes(), valueDeserializer), handler.getLastResultBytes()); +} else { +return null; +} +} + +@Override +public boolean replace(AtomicCacheEntry entry, Serializer keySerializer, Serializer valueSerializer) throws IOException { +final byte[] rowIdBytes = serialize(entry.getKey(), keySerializer); +final byte[] valueBytes = serialize(entry.getValue(), valueSerializer); +final byte[] revision = entry.getRevision().orElse(null); +final PutColumn putColumn = new PutColumn(hBaseColumnFamilyBytes, hBaseColumnQualifierBytes, valueBytes); + +// If the current revision is unset then only insert the row if it doesn't already exist. +return hBaseCl
[nifi] branch master updated: Renew my key.
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 9158498 Renew my key. 9158498 is described below commit 91584987198b2c2c6060f324df8ce79f38d16226 Author: Koji Kawamura AuthorDate: Wed Jun 12 14:48:22 2019 +0900 Renew my key. Signed-off-by: Koji Kawamura --- KEYS | 118 +-- 1 file changed, 59 insertions(+), 59 deletions(-) diff --git a/KEYS b/KEYS index f2ab0ae..ed77dba 100644 --- a/KEYS +++ b/KEYS @@ -780,65 +780,6 @@ G9p0uNUNjZa6XC14K9XK54ar0yyjLg== =jmYd -END PGP PUBLIC KEY BLOCK- -pub 4096R/C89E4758 2016-07-20 [expires: 2017-07-20] -uid [ultimate] Koji Kawamura (CODE SIGNING KEY) -sig 3C89E4758 2016-07-20 Koji Kawamura (CODE SIGNING KEY) -sub 4096R/54AFE8EE 2016-07-20 [expires: 2017-07-20] -sig C89E4758 2016-07-20 Koji Kawamura (CODE SIGNING KEY) - --BEGIN PGP PUBLIC KEY BLOCK- -Version: GnuPG v2 - -mQINBFePFGoBEADhX2gMx2YEkW2rQyfM5qgXDvOYt2fqIwYnKktMa2uiUNWulszZ -dyYEhfbBIIj4vnr5BvIw/o+aq91FiyUfv56x+eH+JB/589DvcHq+VO37+vuTJlK+ -Lp6yBftgQBl7m0kxjadtLwF8gUNwkJ2H1VE0LbANLyue3SsFqBf6dJR4aCxyzVr2 -QyoCMIxXhFJfwf//oz91iW0lDVjQvcFyJGNtuVTK5dQzY5duSmTa8qI8qhsRZmgh -pueXoZCepd5IfZnTLDftIezifSSYHjNQddOKhXIyi1x6EwUkh6qw6nkKFRus2aKr -/sCDT6BTFqH7H5EnNNCYBjlkXUQJ51s/BJ8NpHMVJNxMiR3F/2E2AfwE+JQJtkM0 -Uwh4Y5qXR46QHIewM2r2Ao/IgDcbVH34Xz6i3piVdtf4SAMCB1jEkYmfUNGX0zSS -qbxvLA3lFp4BpoR1y0uwjlHIlrSoNe5YDNUTVJ7+d/XJOZ6koUaOK4twJc0x8V0O -U+QgrGKWcFJWtdLLbGEyZAZYP+7HqlzcttIvkY9gE5WQ+8P6M13uJrkApQMYKbOw -FkVppsHMqb8l8xtwP2b1Ax84siEcxNLi0GBPnnlHvX33063FJvrrpmBwpfF8zB5P -/k15nxRua3MqjRBeiiAeU6kz2OHs79gNMZkUFIX+AyD5qG8ZA85Pmz++BwARAQAB -tDpLb2ppIEthd2FtdXJhIChDT0RFIFNJR05JTkcgS0VZKSA8aWpva2FydW1hd2Fr -QGFwYWNoZS5vcmc+iQI9BBMBCgAnBQJXjxRqAhsDBQkB4TOABQsJCAcDBRUKCQgL -BRYCAwEAAh4BAheAAAoJEDYTaw7InkdYiKwQAIHkmIlGCRfmewxZ6M8KWTSyGjj7 -6iKZ+Znuzbs1iKqNannOWKOn1rfde01bsoKvwrDyamj4KHLkXy59AClfKZJXECYa -o025P6ktnM+t+kj2VCLhEgKY7N67SkQ/vaRNJGMpRlWSaMSrjptu7RCHXqEit8BZ -ttcet2/A6t9Qnco1LZkzTkkyDrVeO7sSJi/M3NTikaDE7ylF+re80mzP5QaR04g+ -8Rdjv3g9TNd5DIELWtepvLwCwKNRvsjN8fPL+LQenFvagM/1Vn9L7Lfz3kOhJC1j -ZKFn1xpBxc59pwE3yy5DNAEspQ260AXoDSiDHc6RsBcMc7EyJakzspbmyHv26QeV -X/E3JpC1eEvtGiphWs95CZc+GYdbKimYbb5Ughca4Scb+3Q67Ye5eA70Sl55dMCq -2dX7qFYWcmPAvz8e7hkQxnwQyIQ9IwsgaSZV3U638ds1/8KcmOor2N9tMx/Akww2 -jw9uIDJCjZfoElpAjyF3CbYIDKHvYsAYp0Y1e01xSoC6xKMEPtxFMsWmDpi96Tis -RmVQMOvRGDrV0b5BfzywrqFv4F4PHgil31PRG21nomt5aWdyuM7nrkkfiuMOoxz2 -9DAlxpY7qrURLJd3xvAauUq4Ml4bR8hTsoJypQcRlpb14z1Fyw2FuSsOWRftsEx/ -6lr3RTIM1P8g22nYuQINBFePFGoBEADgHXXSeLRODMw9QlNe6kSprDDmSrLiUplv -XdN0rob1vYutUD3G6OhMQ5NYBHH125UKmC1Vs+xUHZ/WILNBgCznnzL+YJBsjR3P -Cjx4l5NFQNOFPtRR+l67QmpKlZsXnfXZRXud1xdpFWohBkz6AJdTC0emsQ70N6WZ -S/h8wG6/PTsjz172LYAkQ05ARNWHUsneGfMEV/TZjTy3fIipWFMLX3sM3mkXl21w -UgXDsjoJiS1gWhoXBUg7/oavs9SO09ZCg8gVfJvezQqyu8c0KZpmYQCzRT46KE+T -6VP37dfQY8jkXKLD1gWjMsnGGngA0M74+pwdGj4McyeKBSYhbryS4QfIGVEf+K2z -Yph8FDr+t2DNXH04XWj7Z9jlYjO8faiHtfQjEIbKNprHC6Zo5/fyQFS2qKjJVhae -oIiQf/zMw59IwrCUu8Q49sCf0iHGaNUHSN7q4OUDE5oeoafFqYZl9WGtTEyKyb9i -KOUFaku9qAuRDm9uhDtbbB7oOyFlIDO6jBBT54U6b1PqQ0DVNJHP7EQyEXdP8TsE -yd3T4DfC8RVd1ZEeCb+Eif9Wfjz0+SPFz0kV6f4dEEZ5gw/n4soZuS5ZH2ae4CVp -H2rIzPqHqA12bJ/Z3DPdqdVE9YKrjanhp6wODSU9WL1Zq5J9RbkixHkIiMDJUpvY -pjJUJCRDCQARAQABiQIlBBgBCgAPBQJXjxRqAhsMBQkB4TOAAAoJEDYTaw7InkdY -fAUQAN++Y2ENGP6EAsLQuuhv7U5269EHHAXq/z4/qVDbPqjp//0Z9l/34CE64SUO -W1+5jFuzFNjYkL4029dY/NudP7L3gtYYF2eBQ3qmPCePsXbo2889ByvoaCSmX2MF -MhNg7DPu/1xXTPjxYgnFget5/ad2GkJQbKfy6mKddIQrFm0SKXPP20fDwXsDQ+Q9 -e2946jVqg3isPSq/glP7nj7bs7KmFQ6mA+tNfolR450o+PV+3Al+xlmMvGn4wSRf -ZBtlEUcsScv6i8vGPwnXmUu11NED9MjjrJj0G18uwfHzXMQzbDCHWy+rqX4w65m7 -4pIrcTyYiVMTg5sCdUTMUYi9hzWwE4AT0vee4h4+jY8TJxawJC/f12kig2U9618A -ZsSrLLBJYh6rsIexmmhITNkRnnYU4sUyAPtvuTdXU7VagRyeUyB7+Q7UATpHDNWh -lu6rTLFZ5TYqBz8vS/h/pBoSJSDfpTXGSjUUxJrjYEBpmD8hDIDlA1iO8GYqb442 -BzezmX5TWerF5OgeZCh5CtD/XRwubt7esaUGKcaQrMqV6YUU5CfgLMIk3GFHuvWR -4K1qvBpRNt2chQuAsl14Tu+jND2klcgkw1QVmY4yLTbXuna7sJKlozA7ZcTxJGh9 -XSQLfHE6JkMV7M6Y5ObosaDPZ0UAKl8XbZjwkc2Nf+fZoITI -=Gsga --END PGP PUBLIC KEY BLOCK- - pub 4096R/50711C39 2016-08-03 [expires: 2020-08-03] uid [ultimate] Bryan Bende sig 350711C39 2016-08-03 Bryan Bende @@ -1146,3 +1087,62 @@ Zqi3ByKJm13bffocp/6ofBjEEr9dZktYERDu7BLtatHgW5ynYilkBz71JqChFjad WYVLW4kO81Y= =9ayN -END PGP PUBLIC KEY BLOCK- + +pub 4096R/C89E4758 2016-07-20 [expires: 2022-06-11] +uid [ultimate] Koji Kawamura (CODE SIGNING KEY) +sig 3C89E4758 2019-06-12 Koji Kawamura (CODE SIGNING KEY) +sub 4096R/54AFE8EE 2016-07-20 [expires: 2022-06-11] +sig C89E4758 2019-06-12 Koji Kawamura (CODE SIGNING KEY) + +-BEGIN PGP PUBLIC KEY BLOCK- +Version: GnuPG v2
[nifi] branch master updated: NIFI-6370: Allow multiple SQL statements in PutDatabaseRecord
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 05f3cad NIFI-6370: Allow multiple SQL statements in PutDatabaseRecord 05f3cad is described below commit 05f3cadee8ee76aa05a1d35ffb4598ddeca4f549 Author: Matthew Burgess AuthorDate: Mon Jun 10 12:07:44 2019 -0400 NIFI-6370: Allow multiple SQL statements in PutDatabaseRecord This closes #3528. Signed-off-by: Koji Kawamura --- .../processors/standard/PutDatabaseRecord.java | 34 +- .../standard/TestPutDatabaseRecord.groovy | 77 +- 2 files changed, 107 insertions(+), 4 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java index d79cf3c..3046cbf 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java @@ -239,6 +239,17 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .build(); +static final PropertyDescriptor ALLOW_MULTIPLE_STATEMENTS = new PropertyDescriptor.Builder() +.name("put-db-record-allow-multiple-statements") +.displayName("Allow Multiple SQL Statements") +.description("If the Statement Type is 'SQL' (as set in the statement.type attribute), this field indicates whether to split the field value by a semicolon and execute each statement " ++ "separately. If any statement causes an error, the entire set of statements will be rolled back. If the Statement Type is not 'SQL', this field is ignored.") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.required(true) +.allowableValues("true", "false") +.defaultValue("false") +.build(); + static final PropertyDescriptor QUOTED_IDENTIFIERS = new PropertyDescriptor.Builder() .name("put-db-record-quoted-identifiers") .displayName("Quote Column Identifiers") @@ -309,6 +320,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { pds.add(UNMATCHED_COLUMN_BEHAVIOR); pds.add(UPDATE_KEYS); pds.add(FIELD_CONTAINING_SQL); +pds.add(ALLOW_MULTIPLE_STATEMENTS); pds.add(QUOTED_IDENTIFIERS); pds.add(QUOTED_TABLE_IDENTIFIER); pds.add(QUERY_TIMEOUT); @@ -404,7 +416,15 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { getLogger().warn("Failed to process {} due to {}", new Object[]{inputFlowFile, e}, e); -if (e instanceof BatchUpdateException) { +// Check if there was a BatchUpdateException or if multiple SQL statements were being executed and one failed +final String statementTypeProperty = context.getProperty(STATEMENT_TYPE).getValue(); +String statementType = statementTypeProperty; +if (USE_ATTR_TYPE.equals(statementTypeProperty)) { +statementType = inputFlowFile.getAttribute(STATEMENT_TYPE_ATTRIBUTE); +} + +if (e instanceof BatchUpdateException +|| (SQL_TYPE.equalsIgnoreCase(statementType) && context.getProperty(ALLOW_MULTIPLE_STATEMENTS).asBoolean())) { try { // Although process session will move forward in order to route the failed FlowFile, // database transaction should be rolled back to avoid partial batch update. @@ -567,8 +587,16 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { throw new MalformedRecordException(format("Record had no (or null) value for Field Containing SQL: %s, FlowFile %s", sqlField, flowFile)); } -// Execute the statement as-is -s.execute((String) sql); +// Execute the statement(s) as-is +if (context.getProperty(ALLOW_MULTIPLE_STATEMENTS).asBoolean()) { +String regex = "(?
[nifi] branch master updated: NIFI-6361: Add fix to PutFile processor
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 1975101 NIFI-6361: Add fix to PutFile processor 1975101 is described below commit 1975101292fcd0fceb0d30594da54e5257bc7700 Author: Andres Garagiola AuthorDate: Fri Jun 7 09:54:44 2019 -0300 NIFI-6361: Add fix to PutFile processor When PutFile uses 'replace' conflict resolution and max files, there is an issue when the folder has X files, and the limit is also X. The processor fails instead of replacing it, leaving X files. This commit fixes that issue. This closes #3524. Signed-off-by: Andres Garagiola Signed-off-by: Koji Kawamura --- .../apache/nifi/processors/standard/PutFile.java | 15 +- .../nifi/processors/standard/TestPutFile.java | 210 + 2 files changed, 222 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java index 0c95c75..8784c75 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java @@ -49,6 +49,7 @@ import java.nio.file.attribute.PosixFilePermissions; import java.nio.file.attribute.UserPrincipalLookupService; import java.text.DateFormat; import java.text.SimpleDateFormat; +import java.util.Arrays; import java.util.ArrayList; import java.util.Collections; import java.util.Date; @@ -203,8 +204,9 @@ public class PutFile extends AbstractProcessor { Path tempDotCopyFile = null; try { final Path rootDirPath = configuredRootDirPath; -final Path tempCopyFile = rootDirPath.resolve("." + flowFile.getAttribute(CoreAttributes.FILENAME.key())); -final Path copyFile = rootDirPath.resolve(flowFile.getAttribute(CoreAttributes.FILENAME.key())); +String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key()); +final Path tempCopyFile = rootDirPath.resolve("." + filename); +final Path copyFile = rootDirPath.resolve(filename); if (!Files.exists(rootDirPath)) { if (context.getProperty(CREATE_DIRS).asBoolean()) { @@ -224,7 +226,7 @@ public class PutFile extends AbstractProcessor { final Path finalCopyFileDir = finalCopyFile.getParent(); if (Files.exists(finalCopyFileDir) && maxDestinationFiles != null) { // check if too many files already -final int numFiles = finalCopyFileDir.toFile().list().length; +final long numFiles = getFilesNumberInFolder(finalCopyFileDir, filename); if (numFiles >= maxDestinationFiles) { flowFile = session.penalize(flowFile); @@ -336,6 +338,13 @@ public class PutFile extends AbstractProcessor { } } +private long getFilesNumberInFolder(Path folder, String filename) { +String[] filesInFolder = folder.toFile().list(); +return Arrays.stream(filesInFolder) +.filter(eachFilename -> !eachFilename.equals(filename)) +.count(); +} + protected String stringPermissions(String perms) { String permissions = ""; final Pattern rwxPattern = Pattern.compile("^[rwx-]{9}$"); 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 new file mode 100644 index 000..51ad7c0 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutFile.java @@ -0,0 +1,210 @@ +/* + * 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
[nifi] branch master updated: NIFI-6349 Fix to MergeRecords when handling fragments over multiple iterations
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 99350b7 NIFI-6349 Fix to MergeRecords when handling fragments over multiple iterations 99350b7 is described below commit 99350b761dda3458757563b633ed21e5524257d4 Author: Evan Reynolds AuthorDate: Tue Jun 4 17:34:26 2019 -0700 NIFI-6349 Fix to MergeRecords when handling fragments over multiple iterations Fixed fragment count attribute check This closes #3517. Signed-off-by: Koji Kawamura --- .../nifi/processors/standard/merge/RecordBin.java | 30 ++ .../standard/merge/RecordBinManager.java | 11 +-- .../nifi/processors/standard/TestMergeRecord.java | 36 ++ 3 files changed, 53 insertions(+), 24 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java index a96e119..139b2a4 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java @@ -203,6 +203,10 @@ public class RecordBin { return false; } +if(thresholds.getFragmentCountAttribute().isPresent() && this.fragmentCount == getMinimumRecordCount()) { +return true; +} + int maxRecords = thresholds.getMaxRecords(); if (recordCount >= maxRecords) { @@ -213,22 +217,6 @@ public class RecordBin { return true; } -Optional fragmentCountAttribute = thresholds.getFragmentCountAttribute(); -if(fragmentCountAttribute != null && fragmentCountAttribute.isPresent()) { -final Optional fragmentCountValue = flowFiles.stream() -.filter(ff -> ff.getAttribute(fragmentCountAttribute.get()) != null) -.map(ff -> ff.getAttribute(fragmentCountAttribute.get())) -.findFirst(); -if (fragmentCountValue.isPresent()) { -try { -int expectedFragments = Integer.parseInt(fragmentCountValue.get()); -if (this.fragmentCount == expectedFragments) -return true; -} catch (NumberFormatException nfe) { -this.logger.error(nfe.getMessage(), nfe); -} -} -} return false; } finally { readLock.unlock(); @@ -349,14 +337,14 @@ public class RecordBin { count = Integer.parseInt(countVal); } catch (final NumberFormatException nfe) { logger.error("Could not merge bin with {} FlowFiles because the '{}' attribute had a value of '{}' for {} but expected a number", -new Object[] {flowFiles.size(), countAttr.get(), countVal, flowFile}); +new Object[] {flowFiles.size(), countAttr.get(), countVal, flowFile}); fail(); return; } if (expectedBinCount != null && count != expectedBinCount) { logger.error("Could not merge bin with {} FlowFiles because the '{}' attribute had a value of '{}' for {} but another FlowFile in the bin had a value of {}", -new Object[] {flowFiles.size(), countAttr.get(), countVal, flowFile, expectedBinCount}); +new Object[] {flowFiles.size(), countAttr.get(), countVal, flowFile, expectedBinCount}); fail(); return; } @@ -366,15 +354,15 @@ public class RecordBin { if (expectedBinCount == null) { logger.error("Could not merge bin with {} FlowFiles because the '{}' attribute was not present on any of the FlowFiles", -new Object[] {flowFiles.size(), countAttr.get()}); +new Object[] {flowFiles.size(), countAttr.get()}); fail(); return; } if (expectedBinCount != flowFiles.size()) { logger.error("Could not merge bin with {} FlowFiles because the '{}' attribute had a value of '{}' but o
[nifi] branch master updated: NIFI-6319 Update User Guide for RPG/S2S chnages for multiple URLs, batch settings and remote input/output ports
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 0b8d05a NIFI-6319 Update User Guide for RPG/S2S chnages for multiple URLs, batch settings and remote input/output ports 0b8d05a is described below commit 0b8d05aabd75696118963ac6535fd657b5dd5706 Author: Andrew Lim AuthorDate: Wed May 29 15:24:02 2019 -0400 NIFI-6319 Update User Guide for RPG/S2S chnages for multiple URLs, batch settings and remote input/output ports NIFI-6319 Improved Remote Process Group Ports screenshot This closes #3503. Signed-off-by: Koji Kawamura --- .../main/asciidoc/images/add-input-port-S2S.png| Bin 0 -> 109037 bytes .../main/asciidoc/images/add-output-port-S2S.png | Bin 0 -> 109298 bytes .../main/asciidoc/images/remote-group-anatomy.png | Bin 59409 -> 51949 bytes .../asciidoc/images/remote-group-ports-dialog.png | Bin 22930 -> 95111 bytes .../images/remote-port-connection-status.png | Bin 27577 -> 106748 bytes nifi-docs/src/main/asciidoc/user-guide.adoc| 107 - 6 files changed, 40 insertions(+), 67 deletions(-) diff --git a/nifi-docs/src/main/asciidoc/images/add-input-port-S2S.png b/nifi-docs/src/main/asciidoc/images/add-input-port-S2S.png new file mode 100644 index 000..70a81a1 Binary files /dev/null and b/nifi-docs/src/main/asciidoc/images/add-input-port-S2S.png differ diff --git a/nifi-docs/src/main/asciidoc/images/add-output-port-S2S.png b/nifi-docs/src/main/asciidoc/images/add-output-port-S2S.png new file mode 100644 index 000..e90ebba Binary files /dev/null and b/nifi-docs/src/main/asciidoc/images/add-output-port-S2S.png differ diff --git a/nifi-docs/src/main/asciidoc/images/remote-group-anatomy.png b/nifi-docs/src/main/asciidoc/images/remote-group-anatomy.png index 2ba7d37..66f3f70 100644 Binary files a/nifi-docs/src/main/asciidoc/images/remote-group-anatomy.png and b/nifi-docs/src/main/asciidoc/images/remote-group-anatomy.png differ diff --git a/nifi-docs/src/main/asciidoc/images/remote-group-ports-dialog.png b/nifi-docs/src/main/asciidoc/images/remote-group-ports-dialog.png index f60d012..6a0196a 100644 Binary files a/nifi-docs/src/main/asciidoc/images/remote-group-ports-dialog.png and b/nifi-docs/src/main/asciidoc/images/remote-group-ports-dialog.png differ diff --git a/nifi-docs/src/main/asciidoc/images/remote-port-connection-status.png b/nifi-docs/src/main/asciidoc/images/remote-port-connection-status.png index a543202..f80d6ec 100644 Binary files a/nifi-docs/src/main/asciidoc/images/remote-port-connection-status.png and b/nifi-docs/src/main/asciidoc/images/remote-port-connection-status.png differ diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc index 959b863..2a3a7b8 100644 --- a/nifi-docs/src/main/asciidoc/user-guide.adoc +++ b/nifi-docs/src/main/asciidoc/user-guide.adoc @@ -298,7 +298,7 @@ While the options available from the context menu vary, the following options ar - *Configure*: This option allows the user to establish or change the configuration of the Processor (see <>). -NOTE: For Processors, Ports, Remote Process Groups, Connections and Labels, it is possible to open the configuration dialog by double-clicking on desired component. +NOTE: For Processors, Ports, Remote Process Groups, Connections and Labels, it is possible to open the configuration dialog by double-clicking on the desired component. - *Start* or *Stop*: This option allows the user to start or stop a Processor; the option will be either Start or Stop, depending on the current state of the Processor. - *Enable* or *Disable*: This option allows the user to enable or disable a Processor; the option will be either Enable or Disable, depending on the current state of the Processor. @@ -349,8 +349,8 @@ image:iconProcessGroup.png["Process Group", width=32] and maintain. When a Process Group is dragged onto the canvas, the DFM is prompted to name the Process Group. All Process Groups within the same parent group must have unique names. The Process Group will then be nested within that parent group. -Once you have dragged a Process Group onto the canvas, you can interact with it by right-clicking on the Process Group and selecting an option from -context menu.The options available to you from the context menu vary, depending on the privileges assigned to you. +Once you have dragged a Process Group onto the canvas, you can interact with it by right-clicking on the Process Group and selecting an option from the +context menu. The options available to you from the context menu vary, depending on the privileges assigned to you. image::nifi-process-group-menu.png["Process Group Menu"] @@ -377,21 +377,15 @@ NOTE: It is als
[nifi] branch master updated: NIFI-6218 Support setting transactional.id in PublishKafka/PublishKafkaRecord
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new aefdc93 NIFI-6218 Support setting transactional.id in PublishKafka/PublishKafkaRecord aefdc93 is described below commit aefdc9377cc00c82ab70d44da5effd42da63bb04 Author: Ferenc Szabó AuthorDate: Tue Apr 16 16:49:52 2019 +0200 NIFI-6218 Support setting transactional.id in PublishKafka/PublishKafkaRecord Added new property to the processors. If it is empty, then the behaviour is the same as before. Adde unit test to check if KafkaProducer receives the new property. Testing Done: Unit tests. Connecting to kafka and verifying transactionalId-s in kafka server log. fixing review findings This closes #3439. Signed-off-by: Koji Kawamura --- .../kafka/pubsub/KafkaProcessorUtils.java | 12 +++ .../kafka/pubsub/PublishKafkaRecord_0_11.java | 14 +++- .../processors/kafka/pubsub/PublishKafka_0_11.java | 14 +++- .../processors/kafka/pubsub/PublisherPool.java | 8 +++-- .../kafka/pubsub/KafkaProcessorUtilsTest.java | 41 ++ .../processors/kafka/pubsub/TestPublisherPool.java | 6 ++-- .../kafka/pubsub/KafkaProcessorUtils.java | 12 +++ .../kafka/pubsub/PublishKafkaRecord_1_0.java | 14 +++- .../processors/kafka/pubsub/PublishKafka_1_0.java | 14 +++- .../processors/kafka/pubsub/PublisherPool.java | 8 +++-- .../kafka/pubsub/KafkaProcessorUtilsTest.java | 41 ++ .../processors/kafka/pubsub/TestPublisherPool.java | 6 ++-- .../kafka/pubsub/KafkaProcessorUtils.java | 12 +++ .../kafka/pubsub/PublishKafkaRecord_2_0.java | 14 +++- .../processors/kafka/pubsub/PublishKafka_2_0.java | 14 +++- .../processors/kafka/pubsub/PublisherPool.java | 8 +++-- .../kafka/pubsub/KafkaProcessorUtilsTest.java | 41 ++ .../processors/kafka/pubsub/TestPublisherPool.java | 5 ++- 18 files changed, 258 insertions(+), 26 deletions(-) diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java index 37c6599..9bc4917 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java @@ -25,7 +25,9 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import java.util.regex.Pattern; import org.apache.commons.lang3.StringUtils; @@ -332,6 +334,16 @@ final class KafkaProcessorUtils { } /** + * Method used to create a transactional id Supplier for KafkaProducer + * + * @param prefix String transactional id prefix, can be null + * @return A Supplier that generates transactional id + */ +static Supplier getTransactionalIdSupplier(String prefix) { +return () -> (prefix == null ? "" : prefix) + UUID.randomUUID().toString(); +} + +/** * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259 * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients * diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java index 0815518..cdd37a6 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_0_11.java @@ -62,6 +62,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import java.util.regex.Pattern; @Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "0.11.x"}) @@ -219,6 +220,14 @@ public class PublishKafkaRecord_0_11 extends AbstractProcessor {
[nifi] branch master updated: NIFI-6315 Ensuring remote ports get tracked correctly when saving/retrieving versioned flows
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new e7f4af6 NIFI-6315 Ensuring remote ports get tracked correctly when saving/retrieving versioned flows e7f4af6 is described below commit e7f4af68014ea99f1e351436d6a078ad4b3346a6 Author: Bryan Bende AuthorDate: Thu May 23 14:18:37 2019 -0400 NIFI-6315 Ensuring remote ports get tracked correctly when saving/retrieving versioned flows NIFI-6315 Renaming public ports during import/change-version when name alreadye exists, and ignorning name change from flow diffs NIFI-6315 Ensuring public ports get renamed during copy/paste NIFI-6315 Set max concurrent tasks on Port based on value from VersionedPort This closes #3488. Signed-off-by: Koji Kawamura --- .../apache/nifi/controller/flow/FlowManager.java | 17 .../nifi/controller/StandardFlowSnippet.java | 25 +- .../nifi/controller/flow/StandardFlowManager.java | 21 + .../apache/nifi/groups/StandardProcessGroup.java | 98 +++--- .../flow/mapping/NiFiRegistryFlowMapper.java | 8 ++ .../apache/nifi/util/FlowDifferenceFilters.java| 20 + .../nifi/util/TestFlowDifferenceFilters.java | 45 ++ .../apache/nifi/web/StandardNiFiServiceFacade.java | 5 ++ .../org/apache/nifi/web/api/dto/DtoFactory.java| 5 ++ 9 files changed, 214 insertions(+), 30 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java index c48c06f..f2a9cf9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java @@ -33,6 +33,7 @@ import org.apache.nifi.web.api.dto.FlowSnippetDTO; import java.net.URL; import java.util.Collection; +import java.util.Optional; import java.util.Set; public interface FlowManager { @@ -76,6 +77,22 @@ public interface FlowManager { Set getPublicOutputPorts(); /** + * Gets the public input port with the given name. + * + * @param name the port name + * @return an optional containing the public input port with the given name, or empty if one does not exist + */ +Optional getPublicInputPort(String name); + +/** + * Gets the public output port with the given name. + * + * @param name the port name + * @return an optional containing the public output port with the given name, or empty if one does not exist + */ +Optional getPublicOutputPort(String name); + +/** * Creates a new Remote Process Group with the given ID that points to the given URI * * @param id Remote Process Group ID diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java index 56622f8..d466c9c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java @@ -72,6 +72,7 @@ import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -284,7 +285,8 @@ public class StandardFlowSnippet implements FlowSnippet { for (final PortDTO portDTO : dto.getInputPorts()) { final Port inputPort; if (group.isRootGroup() || Boolean.TRUE.equals(portDTO.getAllowRemoteAccess())) { -inputPort = flowManager.createPublicInputPort(portDTO.getId(), portDTO.getName()); +final String portName = generatePublicInputPortName(flowManager, portDTO.getName()); +inputPort = flowManager.createPublicInputPort(portDTO.getId(), portName); if (portDTO.getGroupAccessControl() != null) { ((PublicPort) inputPort).setGroupAccessControl(portDTO.getGroupAccessControl()); } @@ -308,7 +310,8 @@ public class StandardFlowSnippet implements FlowSnippet { for (final PortDTO portDTO : dto.getOutputPorts
[nifi-registry] branch master updated: NIFIREG-276 Updating StandardFlowComparator to consider concurrent tasks changes on public ports
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi-registry.git The following commit(s) were added to refs/heads/master by this push: new c8cb270 NIFIREG-276 Updating StandardFlowComparator to consider concurrent tasks changes on public ports c8cb270 is described below commit c8cb2700152ca08078cab0a4d78b7740c40bd731 Author: Bryan Bende AuthorDate: Fri May 31 10:36:06 2019 -0400 NIFIREG-276 Updating StandardFlowComparator to consider concurrent tasks changes on public ports This closes #193. Signed-off-by: Koji Kawamura --- .../registry/flow/diff/StandardFlowComparator.java | 26 +- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java b/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java index f98225e..82c00e5 100644 --- a/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java +++ b/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java @@ -17,15 +17,6 @@ package org.apache.nifi.registry.flow.diff; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; - import org.apache.nifi.registry.flow.VersionedComponent; import org.apache.nifi.registry.flow.VersionedConnection; import org.apache.nifi.registry.flow.VersionedControllerService; @@ -38,6 +29,15 @@ import org.apache.nifi.registry.flow.VersionedPropertyDescriptor; import org.apache.nifi.registry.flow.VersionedRemoteGroupPort; import org.apache.nifi.registry.flow.VersionedRemoteProcessGroup; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + public class StandardFlowComparator implements FlowComparator { private static final String DEFAULT_LOAD_BALANCE_STRATEGY = "DO_NOT_LOAD_BALANCE"; private static final String DEFAULT_PARTITIONING_ATTRIBUTE = ""; @@ -249,7 +249,13 @@ public class StandardFlowComparator implements FlowComparator { } private void compare(final VersionedPort portA, final VersionedPort portB, final Set differences) { -compareComponents(portA, portB, differences); +if (compareComponents(portA, portB, differences)) { +return; +} + +if (portA != null && portA.isAllowRemoteAccess() && portB != null && portB.isAllowRemoteAccess()) { +addIfDifferent(differences, DifferenceType.CONCURRENT_TASKS_CHANGED, portA, portB, VersionedPort::getConcurrentlySchedulableTaskCount); +} } private void compare(final VersionedRemoteProcessGroup rpgA, final VersionedRemoteProcessGroup rpgB, final Set differences) {
[nifi] branch master updated: NIFI-6022 Added try/catch to protect against secondary exception in a primary exception catch clause
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 097b9c3 NIFI-6022 Added try/catch to protect against secondary exception in a primary exception catch clause 097b9c3 is described below commit 097b9c30754c77858aff684b113ed305cec7cf57 Author: slyouts AuthorDate: Wed May 15 15:19:25 2019 -0400 NIFI-6022 Added try/catch to protect against secondary exception in a primary exception catch clause NIFI-6022 mods per PR review NIFI-6022 mods per PR review This closes #3476. Signed-off-by: Koji Kawamura --- .../src/main/java/org/apache/nifi/jms/processors/JMSConsumer.java | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSConsumer.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSConsumer.java index a2c73b4..3c16ee2 100644 --- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSConsumer.java +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSConsumer.java @@ -126,7 +126,12 @@ final class JMSConsumer extends JMSWorker { // We need to call recover to ensure that in the event of // abrupt end or exception the current session will stop message // delivery and restart with the oldest unacknowledged message -session.recover(); +try { +session.recover(); +} catch (Exception e1) { +// likely the session is closed...need to catch this so that the root cause of failure is propagated +processLog.debug("Failed to recover JMS session while handling initial error. The recover error is: ", e1); +} throw e; } finally { JmsUtils.closeMessageConsumer(msgConsumer);
[nifi] branch master updated: NIFI-6304 added trim, toLowerCase and toUpperCase to record path operations.
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 6a06cd3 NIFI-6304 added trim, toLowerCase and toUpperCase to record path operations. 6a06cd3 is described below commit 6a06cd3094eccaa264d575906106504092f95cd6 Author: Mike Thomsen AuthorDate: Thu May 16 21:11:05 2019 -0400 NIFI-6304 added trim, toLowerCase and toUpperCase to record path operations. NIFI-6304 Updated code based on code review. NIFI-6304 Updated documentation. NIFI-6304 Refactored to make it simpler NIFI-6304 Reverted Concat to its last state. This closes #3478. Signed-off-by: Koji Kawamura --- .../apache/nifi/record/path/functions/Concat.java | 2 +- .../{Concat.java => NoArgStringFunction.java} | 46 +-- .../nifi/record/path/functions/ToLowerCase.java| 30 .../nifi/record/path/functions/ToUpperCase.java| 30 .../nifi/record/path/functions/TrimString.java | 30 .../nifi/record/path/paths/RecordPathCompiler.java | 26 +++ .../apache/nifi/record/path/TestRecordPath.java| 90 ++ nifi-docs/src/main/asciidoc/record-path-guide.adoc | 90 +- 8 files changed, 306 insertions(+), 38 deletions(-) diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/Concat.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/Concat.java index daeee05..3b8a3cd 100644 --- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/Concat.java +++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/Concat.java @@ -52,4 +52,4 @@ public class Concat extends RecordPathSegment { return Stream.of(responseValue); } -} +} \ No newline at end of file diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/Concat.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/NoArgStringFunction.java similarity index 51% copy from nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/Concat.java copy to nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/NoArgStringFunction.java index daeee05..8cb801e 100644 --- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/Concat.java +++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/NoArgStringFunction.java @@ -17,39 +17,39 @@ package org.apache.nifi.record.path.functions; -import java.util.stream.Stream; - import org.apache.nifi.record.path.FieldValue; import org.apache.nifi.record.path.RecordPathEvaluationContext; import org.apache.nifi.record.path.StandardFieldValue; import org.apache.nifi.record.path.paths.RecordPathSegment; -import org.apache.nifi.serialization.record.RecordField; -import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.util.DataTypeUtils; -public class Concat extends RecordPathSegment { -private final RecordPathSegment[] valuePaths; +import java.util.stream.Stream; + +/** + * Abstract class for String functions without any argument. + */ +public abstract class NoArgStringFunction extends RecordPathSegment { +private final RecordPathSegment valuePath; -public Concat(final RecordPathSegment[] valuePaths, final boolean absolute) { -super("concat", null, absolute); -this.valuePaths = valuePaths; +public NoArgStringFunction(final String path, final RecordPathSegment valuePath, final boolean absolute) { +super(path, null, absolute); +this.valuePath = valuePath; } @Override -public Stream evaluate(final RecordPathEvaluationContext context) { -Stream concatenated = Stream.empty(); - -for (final RecordPathSegment valuePath : valuePaths) { -final Stream stream = valuePath.evaluate(context); -concatenated = Stream.concat(concatenated, stream); -} - -final StringBuilder sb = new StringBuilder(); -concatenated.forEach(fv -> sb.append(DataTypeUtils.toString(fv.getValue(), (String) null))); - -final RecordField field = new RecordField("concat", RecordFieldType.STRING.getDataType()); -final FieldValue responseValue = new StandardFieldValue(sb.toString(), field, null); -return Stream.of(responseValue); +public Stream evaluate(RecordPathEvaluationContext context) { +return valuePath.evaluate(context).map(fv -> { +final String original = fv.getValue() == null ? "" : DataTypeUtils.toString(fv.getValue(), (String) null); +
[nifi] branch master updated: NIFI-6035 Fix check-style issue
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new c470608 NIFI-6035 Fix check-style issue c470608 is described below commit c470608daf7e5d3267a73fc4b718cd738f17108a Author: Koji Kawamura AuthorDate: Wed May 22 16:07:17 2019 +0900 NIFI-6035 Fix check-style issue --- .../src/test/java/org/apache/nifi/record/path/TestRecordPath.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java b/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java index cd57078..848886f 100644 --- a/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java +++ b/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java @@ -1306,7 +1306,8 @@ public class TestRecordPath { .evaluate(record).getSelectedFields().findFirst().get(); assertEquals("2017-10-20", fieldValue2.getValue()); -final FieldValue fieldValue3 = RecordPath.compile("format( toDate(/date, \"-MM-dd'T'HH:mm:ss'Z'\"), '-MM-dd HH:mm', 'GMT+8:00')").evaluate(record).getSelectedFields().findFirst().get(); +final FieldValue fieldValue3 = RecordPath.compile("format( toDate(/date, \"-MM-dd'T'HH:mm:ss'Z'\"), '-MM-dd HH:mm', 'GMT+8:00')") +.evaluate(record).getSelectedFields().findFirst().get(); assertEquals("2017-10-20 19:00", fieldValue3.getValue()); final FieldValue fieldValueUnchanged = RecordPath.compile("format( toDate(/date, \"-MM-dd'T'HH:mm:ss'Z'\"), 'INVALID' )").evaluate(record).getSelectedFields().findFirst().get();
[nifi] branch master updated: NIFI-6220: If FlowFile is created by cloning a relationship, do not create an ATTRIBUTES_MODIFIED provenance event for it.
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 055b3ca NIFI-6220: If FlowFile is created by cloning a relationship, do not create an ATTRIBUTES_MODIFIED provenance event for it. 055b3ca is described below commit 055b3cac54e3e234c0ee07189b7b7b85e0582d58 Author: Mark Payne AuthorDate: Tue Apr 16 14:52:34 2019 -0400 NIFI-6220: If FlowFile is created by cloning a relationship, do not create an ATTRIBUTES_MODIFIED provenance event for it. NIFI-6220: Updated test name and fixed typo This closes #3438. Signed-off-by: Koji Kawamura --- .../repository/StandardProcessSession.java | 191 +++-- .../integration/provenance/ProvenanceEventsIT.java | 58 +++ 2 files changed, 160 insertions(+), 89 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 14cb70a..c2502d3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -325,6 +325,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE newRecord.setTransferRelationship(record.getTransferRelationship()); // put the mapping into toAdd because adding to records now will cause a ConcurrentModificationException toAdd.put(clone.getId(), newRecord); + +createdFlowFiles.add(newUuid); } } } @@ -639,6 +641,13 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE recordsToSubmit.add(event); addEventType(eventTypesPerFlowFileId, event.getFlowFileUuid(), event.getEventType()); + +final List childUuids = event.getChildUuids(); +if (childUuids != null) { +for (final String childUuid : childUuids) { +addEventType(eventTypesPerFlowFileId, childUuid, event.getEventType()); +} +} } // Finally, add any other events that we may have generated. @@ -684,6 +693,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE if (registeredTypes != null) { if (registeredTypes.get(ProvenanceEventType.CREATE.ordinal()) || registeredTypes.get(ProvenanceEventType.FORK.ordinal()) +|| registeredTypes.get(ProvenanceEventType.CLONE.ordinal()) || registeredTypes.get(ProvenanceEventType.JOIN.ordinal()) || registeredTypes.get(ProvenanceEventType.RECEIVE.ordinal()) || registeredTypes.get(ProvenanceEventType.FETCH.ordinal())) { @@ -771,6 +781,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE provenanceRepo.registerEvents(iterable); } + private void updateEventContentClaims(final ProvenanceEventBuilder builder, final FlowFile flowFile, final StandardRepositoryRecord repoRecord) { final ContentClaim originalClaim = repoRecord.getOriginalClaim(); if (originalClaim == null) { @@ -1678,6 +1689,97 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE } @Override +public FlowFile create(FlowFile parent) { +verifyTaskActive(); +parent = getMostRecent(parent); + +final String uuid = UUID.randomUUID().toString(); + +final Map newAttributes = new HashMap<>(3); +newAttributes.put(CoreAttributes.FILENAME.key(), uuid); +newAttributes.put(CoreAttributes.PATH.key(), DEFAULT_FLOWFILE_PATH); +newAttributes.put(CoreAttributes.UUID.key(), uuid); + +final StandardFlowFileRecord.Builder fFileBuilder = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence()); + +// copy all attributes from parent except for the "special" attributes. Copying the special attributes +// can cause problems -- especially the ALTERNATE_IDENTIFIER, because copying can cause Provenance Events +// to be incorrectly created. +for (final Map.Entry entry : parent.getAttributes().entrySet()) { +final String key = entry.getKey(); +final String value
[nifi] branch master updated: NIFI-6231 - fix source code permissions to be non-executable
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 161e4b5 NIFI-6231 - fix source code permissions to be non-executable 161e4b5 is described below commit 161e4b57632b2d5454390871980288eb66df2502 Author: Dustin Rodrigues AuthorDate: Sat Apr 20 13:10:34 2019 -0400 NIFI-6231 - fix source code permissions to be non-executable This closes #3449. Signed-off-by: Koji Kawamura --- .../src/main/java/org/apache/nifi/avro/AvroTypeUtil.java | 0 .../src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java | 0 .../src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java | 0 .../src/main/java/org/apache/nifi/web/ImageViewerController.java | 0 .../src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java | 0 .../src/main/java/org/apache/nifi/processors/solr/GetSolr.java| 0 .../main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java | 0 .../src/main/java/org/apache/nifi/processors/solr/QuerySolr.java | 0 .../src/main/java/org/apache/nifi/processors/solr/SolrProcessor.java | 0 .../src/main/java/org/apache/nifi/processors/solr/SolrUtils.java | 0 .../src/test/java/org/apache/nifi/processors/solr/QuerySolrIT.java| 0 .../src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java| 0 .../java/org/apache/nifi/processors/solr/TestPutSolrContentStream.java| 0 .../src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java | 0 .../main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java | 0 .../java/org/apache/nifi/processors/standard/TestConvertJSONToSQL.java| 0 .../src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java | 0 .../src/main/java/org/apache/nifi/json/WriteJsonResult.java | 0 .../src/main/java/org/apache/nifi/record/NullSuppression.java | 0 .../src/main/java/org/apache/nifi/xml/ArrayWrapping.java | 0 .../src/main/java/org/apache/nifi/xml/WriteXMLResult.java | 0 .../src/main/java/org/apache/nifi/xml/XMLReader.java | 0 .../src/main/java/org/apache/nifi/xml/XMLRecordReader.java| 0 .../src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java | 0 .../src/test/java/org/apache/nifi/json/TestWriteJsonResult.java | 0 .../src/test/java/org/apache/nifi/xml/TestWriteXMLResult.java | 0 .../src/test/java/org/apache/nifi/xml/TestWriteXMLResultUtils.java| 0 .../src/test/java/org/apache/nifi/xml/TestXMLReader.java | 0 .../src/test/java/org/apache/nifi/xml/TestXMLReaderProcessor.java | 0 .../src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java| 0 .../src/test/java/org/apache/nifi/xml/TestXMLRecordSetWriter.java | 0 .../test/java/org/apache/nifi/xml/TestXMLRecordSetWriterProcessor.java| 0 32 files changed, 0 insertions(+), 0 deletions(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java old mode 100755 new mode 100644 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 old mode 100755 new mode 100644 diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java old mode 100755 new mode 100644 diff --git a/nifi-nar-bundles/nifi-media-bundle/nifi-image-viewer/src/main/java/org/apache/nifi/web/ImageViewerController.java b/nifi-nar-bundles/nifi-media-bundle/nifi-image-viewer/src/main/java/org/apache/nifi/web/ImageViewerController.java old mode 100755 new mode 100644 diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java old mode 100755 new mode 100644 diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java old mode 100755 new mode 100644 diff --git a/nifi-nar
[nifi] branch master updated: NIFI-6198 Infinite recursion in HBase*ClientService
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 49e74b2 NIFI-6198 Infinite recursion in HBase*ClientService 49e74b2 is described below commit 49e74b29dbb3b839af3c793ab266775195c476fd Author: Lars Francke AuthorDate: Tue Apr 9 00:05:52 2019 +0200 NIFI-6198 Infinite recursion in HBase*ClientService This closes #3419. Signed-off-by: Koji Kawamura --- .../src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java | 2 +- .../src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java index a77818a..052b82b 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java @@ -504,7 +504,7 @@ public class HBase_1_1_2_ClientService extends AbstractControllerService impleme @Override public void delete(String tableName, List rowIds) throws IOException { -delete(tableName, rowIds); +delete(tableName, rowIds, null); } @Override diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java index 7d9c843..bc41f76 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java @@ -503,7 +503,7 @@ public class HBase_2_ClientService extends AbstractControllerService implements @Override public void delete(String tableName, List rowIds) throws IOException { -delete(tableName, rowIds); +delete(tableName, rowIds, null); } @Override
[nifi] branch master updated: NIFI-5953 Manage GetTwitter connection retries on '420 Enhance Your Calm' exceptions
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new cded30b NIFI-5953 Manage GetTwitter connection retries on '420 Enhance Your Calm' exceptions cded30b is described below commit cded30b3d2dc0497cf3d06051f55ca304c744250 Author: Kourge <27643232+kourge...@users.noreply.github.com> AuthorDate: Mon Jan 28 14:10:54 2019 +0100 NIFI-5953 Manage GetTwitter connection retries on '420 Enhance Your Calm' exceptions NIFI-5953 Manage GetTwitter connection retries on '420 Enhance Your Calm' exceptions Update "Max Client Error Retries" parameter name. reintriduce client.reconnect() on HTTP_ERROR 420 This closes #3276. Signed-off-by: Koji Kawamura --- .../org/apache/nifi/processors/twitter/GetTwitter.java | 14 ++ .../org/apache/nifi/processors/twitter/TestGetTwitter.java | 5 + 2 files changed, 19 insertions(+) diff --git a/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java b/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java index f0a8b0c..dafa9d0 100644 --- a/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java +++ b/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java @@ -85,6 +85,17 @@ public class GetTwitter extends AbstractProcessor { .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_FIREHOSE, ENDPOINT_FILTER) .defaultValue(ENDPOINT_SAMPLE.getValue()) .build(); +public static final PropertyDescriptor MAX_CLIENT_ERROR_RETRIES = new PropertyDescriptor.Builder() +.name("max-client-error-retries") +.displayName("Max Client Error Retries") +.description("The maximum number of retries to attempt when client experience retryable connection errors." ++ " Client continues attempting to reconnect using an exponential back-off pattern until it successfully reconnects" ++ " or until it reaches the retry limit." ++" It is recommended to raise this value when client is getting rate limited by Twitter API. Default value is 5.") +.required(true) +.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) +.defaultValue("5") +.build(); public static final PropertyDescriptor CONSUMER_KEY = new PropertyDescriptor.Builder() .name("Consumer Key") .description("The Consumer Key provided by Twitter") @@ -161,6 +172,7 @@ public class GetTwitter extends AbstractProcessor { protected void init(final ProcessorInitializationContext context) { final List descriptors = new ArrayList<>(); descriptors.add(ENDPOINT); +descriptors.add(MAX_CLIENT_ERROR_RETRIES); descriptors.add(CONSUMER_KEY); descriptors.add(CONSUMER_SECRET); descriptors.add(ACCESS_TOKEN); @@ -222,6 +234,7 @@ public class GetTwitter extends AbstractProcessor { @OnScheduled public void onScheduled(final ProcessContext context) { final String endpointName = context.getProperty(ENDPOINT).getValue(); +final int maxRetries = context.getProperty(MAX_CLIENT_ERROR_RETRIES).asInteger().intValue(); final Authentication oauth = new OAuth1(context.getProperty(CONSUMER_KEY).getValue(), context.getProperty(CONSUMER_SECRET).getValue(), context.getProperty(ACCESS_TOKEN).getValue(), @@ -319,6 +332,7 @@ public class GetTwitter extends AbstractProcessor { } clientBuilder.hosts(host).endpoint(streamingEndpoint); +clientBuilder.retries(maxRetries); client = clientBuilder.build(); client.connect(); } diff --git a/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/test/java/org/apache/nifi/processors/twitter/TestGetTwitter.java b/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/test/java/org/apache/nifi/processors/twitter/TestGetTwitter.java index e56be67..6124f34 100644 --- a/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/test/java/org/apache/nifi/processors/twitter/TestGetTwitter.java +++ b/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/test/java/org/apache/nifi/processors/twitter/TestGetTwitter.java @@ -26,6 +26,7 @@ public class TestGetTwitter { public void testLocationValidatorWithValidLocations() { final
[nifi] 01/02: NIFI-5979 : enhanced ReplaceText processor with "Number of Occurrences" and "Occurrence offset" configurations
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit d222f14a9e9962fac1953180dfb4c0fe2bec524f Author: pushpavanthar AuthorDate: Sun Mar 17 22:19:37 2019 +0530 NIFI-5979 : enhanced ReplaceText processor with "Number of Occurrences" and "Occurrence offset" configurations Fixed indentation errors to pass checkstyle-checks Added Evaluation Modes as per discussion in PR thread Adding exclusions of test files on rat plugin Added new property 'Line-by-Line Evaluation Mode' and refactored common code This closes #3375. Signed-off-by: Koji Kawamura --- .../nifi-standard-processors/pom.xml | 8 + .../nifi/processors/standard/ReplaceText.java | 369 +++- .../nifi/processors/standard/TestReplaceText.java | 374 + .../LiteralReplaceExceptFirstLine.txt | 11 + .../LiteralReplaceExceptLastLine.txt | 11 + .../LiteralReplaceFirstLine.txt| 11 + .../LiteralReplaceLastLine.txt | 11 + .../ReplaceExceptFirstLine.txt | 11 + .../ReplaceExceptLastLine.txt | 11 + .../TestReplaceTextLineByLine/ReplaceFirstLine.txt | 11 + .../TestReplaceTextLineByLine/ReplaceLastLine.txt | 11 + 11 files changed, 680 insertions(+), 159 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 f2d3d70..03e9bb4 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 @@ -429,6 +429,14 @@ src/test/resources/TestReplaceTextLineByLine/testFile.txt src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt + src/test/resources/TestReplaceTextLineByLine/ReplaceLastLine.txt + src/test/resources/TestReplaceTextLineByLine/ReplaceFirstLine.txt + src/test/resources/TestReplaceTextLineByLine/ReplaceExceptLastLine.txt + src/test/resources/TestReplaceTextLineByLine/ReplaceExceptFirstLine.txt + src/test/resources/TestReplaceTextLineByLine/LiteralReplaceLastLine.txt + src/test/resources/TestReplaceTextLineByLine/LiteralReplaceFirstLine.txt + src/test/resources/TestReplaceTextLineByLine/LiteralReplaceExceptLastLine.txt + src/test/resources/TestReplaceTextLineByLine/LiteralReplaceExceptFirstLine.txt src/test/resources/TestReplaceTextWithMapping/color-fruit-backreference-mapping.txt src/test/resources/TestReplaceTextWithMapping/color-fruit-blank-mapping.txt src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt 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 c6aec0c..851770e 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 @@ -53,6 +53,7 @@ import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.stream.io.util.LineDemarcator; import org.apache.nifi.util.StopWatch; +import javax.annotation.Nullable; import java.io.BufferedWriter; import java.io.IOException; import java.io.InputStream; @@ -85,6 +86,11 @@ public class ReplaceText extends AbstractProcessor { // Constants public static final String LINE_BY_LINE = "Line-by-Line"; +public static final String ALL = "All"; +public static final String FIRST_LINE = "First-Line"; +public static final String EXCEPT_FIRST_LINE = "Except-First-Line"; +public static final String LAST_LINE = "Last-Line"; +public static final String EXCEPT_LAST_LINE = "Except-Last-Line"; public static final String ENTIRE_TEXT = "Entire text"; public static final String prependValue = "Prepend"; public static final String appendValue = "Append"; @@ -99,10 +105,14 @@ public class ReplaceText extends AbstractProcessor { // Properties PREPEND, APPEND, REGEX_REPLACE, LITERAL_REPLA
[nifi] 02/02: NIFI-5979 Add Line-by-Line Evaluation Mode to ReplaceText
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit 1c588f10b276dc8d2937b2c78caf0e034bc43fa8 Author: Koji Kawamura AuthorDate: Mon Apr 8 11:35:30 2019 +0900 NIFI-5979 Add Line-by-Line Evaluation Mode to ReplaceText Refactored to use functions to better handle strategy specific variables via closure. --- .../nifi/processors/standard/ReplaceText.java | 233 ++--- 1 file changed, 105 insertions(+), 128 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 851770e..773458b 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 @@ -53,7 +53,6 @@ import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.stream.io.util.LineDemarcator; import org.apache.nifi.util.StopWatch; -import javax.annotation.Nullable; import java.io.BufferedWriter; import java.io.IOException; import java.io.InputStream; @@ -386,31 +385,28 @@ public class ReplaceText extends AbstractProcessor { } }); } else { -flowFile = session.write(flowFile, new StreamReplaceCallback(this, charset, maxBufferSize, context, flowFile, null)); -} +flowFile = session.write(flowFile, new StreamReplaceCallback(charset, maxBufferSize, context.getProperty(LINE_BY_LINE_EVALUATION_MODE).getValue(), +((bw, oneLine) -> { +// We need to determine what line ending was used and use that after our replacement value. +lineEndingBuilder.setLength(0); +for (int i = oneLine.length() - 1; i >= 0; i--) { +final char c = oneLine.charAt(i); +if (c == '\r' || c == '\n') { +lineEndingBuilder.append(c); +} else { +break; +} +} -return flowFile; -} +bw.write(replacementValue); -public void replaceInLine(BufferedWriter bw, String oneLine, @Nullable Matcher matcher, @Nullable Pattern searchPattern, ProcessContext context, FlowFile flowFile) throws IOException { -final String replacementValue = context.getProperty(REPLACEMENT_VALUE).evaluateAttributeExpressions(flowFile).getValue(); -final StringBuilder lineEndingBuilder = new StringBuilder(2); -// We need to determine what line ending was used and use that after our replacement value. -lineEndingBuilder.setLength(0); -for (int i = oneLine.length() - 1; i >= 0; i--) { -final char c = oneLine.charAt(i); -if (c == '\r' || c == '\n') { -lineEndingBuilder.append(c); -} else { -break; -} +// Preserve original line endings. Reverse string because we iterated over original line ending in reverse order, appending to builder. +// So if builder has multiple characters, they are now reversed from the original string's ordering. +bw.write(lineEndingBuilder.reverse().toString()); +}))); } -bw.write(replacementValue); - -// Preserve original line endings. Reverse string because we iterated over original line ending in reverse order, appending to builder. -// So if builder has multiple characters, they are now reversed from the original string's ordering. -bw.write(lineEndingBuilder.reverse().toString()); +return flowFile; } @Override @@ -433,7 +429,8 @@ public class ReplaceText extends AbstractProcessor { } }); } else { -flowFile = session.write(flowFile, new StreamReplaceCallback(this, charset, maxBufferSize, context, flowFile, null)); +flowFile = session.write(flowFile, new StreamReplaceCallback(charset, maxBufferSize, context.getProperty(LINE_BY_LINE_EVALUATION_MODE).getValue(), +(bw, oneLine) -> bw.write(replacementValue.concat(oneLine; } return flowFile; } @@ -443,11 +440,6 @@ public class ReplaceText extends AbstractProcessor {
[nifi] branch master updated (25cb29e -> 1c588f1)
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from 25cb29e NIFI-6177 Refactor HBaseListLookupService tests to remove use of TestRecordLookupProcessor new d222f14 NIFI-5979 : enhanced ReplaceText processor with "Number of Occurrences" and "Occurrence offset" configurations new 1c588f1 NIFI-5979 Add Line-by-Line Evaluation Mode to ReplaceText The 2 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: .../nifi-standard-processors/pom.xml | 8 + .../nifi/processors/standard/ReplaceText.java | 334 +- .../nifi/processors/standard/TestReplaceText.java | 374 + .../LiteralReplaceExceptFirstLine.txt} | 7 +- .../LiteralReplaceExceptLastLine.txt} | 7 +- .../LiteralReplaceFirstLine.txt} | 3 +- .../LiteralReplaceLastLine.txt}| 1 + .../ReplaceExceptFirstLine.txt}| 7 +- .../{$1$1.txt => ReplaceExceptLastLine.txt}| 6 +- .../ReplaceFirstLine.txt} | 3 +- .../ReplaceLastLine.txt} | 1 + 11 files changed, 584 insertions(+), 167 deletions(-) copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/{TestModifyBytes/noFooter.txt => TestReplaceTextLineByLine/LiteralReplaceExceptFirstLine.txt} (62%) mode change 100755 => 100644 copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/{TestModifyBytes/noFooter.txt => TestReplaceTextLineByLine/LiteralReplaceExceptLastLine.txt} (57%) mode change 100755 => 100644 copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/{TestModifyBytes/noFooter.txt => TestReplaceTextLineByLine/LiteralReplaceFirstLine.txt} (85%) mode change 100755 => 100644 copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/{TestModifyBytes/noFooter.txt => TestReplaceTextLineByLine/LiteralReplaceLastLine.txt} (90%) mode change 100755 => 100644 copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/{TestModifyBytes/noFooter.txt => TestReplaceTextLineByLine/ReplaceExceptFirstLine.txt} (63%) mode change 100755 => 100644 copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/{$1$1.txt => ReplaceExceptLastLine.txt} (88%) mode change 100755 => 100644 copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/{TestModifyBytes/noFooter.txt => TestReplaceTextLineByLine/ReplaceFirstLine.txt} (86%) mode change 100755 => 100644 copy nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/{TestModifyBytes/noFooter.txt => TestReplaceTextLineByLine/ReplaceLastLine.txt} (91%) mode change 100755 => 100644
[nifi] branch master updated: NIFI-6191 HBase 2 ClientService documentation refers to HBase 1.x
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new d372318 NIFI-6191 HBase 2 ClientService documentation refers to HBase 1.x d372318 is described below commit d372318d479c8943a10015e83ef017622731761a Author: Lars Francke AuthorDate: Thu Apr 4 23:55:28 2019 +0200 NIFI-6191 HBase 2 ClientService documentation refers to HBase 1.x This closes #3409. Signed-off-by: Koji Kawamura --- .../src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java index 0f1b2d9..7d9c843 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_2-client-service-bundle/nifi-hbase_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_2_ClientService.java @@ -81,8 +81,7 @@ import java.util.concurrent.atomic.AtomicReference; @RequiresInstanceClassLoading @Tags({ "hbase", "client"}) -@CapabilityDescription("Implementation of HBaseClientService using the HBase 1.1.x client. Although this service was originally built with the 1.1.2 " + -"client and has 1_1_2 in it's name, the client library has since been upgraded to 1.1.13 to leverage bug fixes. This service can be configured " + +@CapabilityDescription("Implementation of HBaseClientService using the HBase 2.1.1 client. This service can be configured " + "by providing a comma-separated list of configuration files, or by specifying values for the other properties. If configuration files " + "are provided, they will be loaded first, and the values of the additional properties will override the values from " + "the configuration files. In addition, any user defined properties on the processor will also be passed to the HBase " +
[nifi] branch master updated: NIFI-6185: ListDatabaseTables processor doesn't close ResultSets
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 73571ae NIFI-6185: ListDatabaseTables processor doesn't close ResultSets 73571ae is described below commit 73571ae30933063b121f9a757a64cf5f7f9c500e Author: Lars Francke AuthorDate: Wed Apr 3 22:53:39 2019 +0200 NIFI-6185: ListDatabaseTables processor doesn't close ResultSets This closes #3405. Signed-off-by: Koji Kawamura --- .../processors/standard/ListDatabaseTables.java| 136 +++-- 1 file changed, 71 insertions(+), 65 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListDatabaseTables.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListDatabaseTables.java index ec2d3c1..fa7dc5c 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListDatabaseTables.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListDatabaseTables.java @@ -230,79 +230,85 @@ public class ListDatabaseTables extends AbstractProcessor { try (final Connection con = dbcpService.getConnection(Collections.emptyMap())) { DatabaseMetaData dbMetaData = con.getMetaData(); -ResultSet rs = dbMetaData.getTables(catalog, schemaPattern, tableNamePattern, tableTypes); -while (rs.next()) { -final String tableCatalog = rs.getString(1); -final String tableSchema = rs.getString(2); -final String tableName = rs.getString(3); -final String tableType = rs.getString(4); -final String tableRemarks = rs.getString(5); +try (ResultSet rs = dbMetaData.getTables(catalog, schemaPattern, tableNamePattern, tableTypes)) { +while (rs.next()) { +final String tableCatalog = rs.getString(1); +final String tableSchema = rs.getString(2); +final String tableName = rs.getString(3); +final String tableType = rs.getString(4); +final String tableRemarks = rs.getString(5); -// Build fully-qualified name -String fqn = Stream.of(tableCatalog, tableSchema, tableName) -.filter(segment -> !StringUtils.isEmpty(segment)) -.collect(Collectors.joining(".")); +// Build fully-qualified name +String fqn = Stream.of(tableCatalog, tableSchema, tableName) + .filter(segment -> !StringUtils.isEmpty(segment)) + .collect(Collectors.joining(".")); -String lastTimestampForTable = stateMapProperties.get(fqn); -boolean refreshTable = true; -try { -// Refresh state if the interval has elapsed -long lastRefreshed = -1; -final long currentTime = System.currentTimeMillis(); -if (!StringUtils.isEmpty(lastTimestampForTable)) { -lastRefreshed = Long.parseLong(lastTimestampForTable); -} -if (lastRefreshed == -1 || (refreshInterval > 0 && currentTime >= (lastRefreshed + refreshInterval))) { -stateMapProperties.remove(lastTimestampForTable); -} else { -refreshTable = false; +String lastTimestampForTable = stateMapProperties.get(fqn); +boolean refreshTable = true; +try { +// Refresh state if the interval has elapsed +long lastRefreshed = -1; +final long currentTime = System.currentTimeMillis(); +if (!StringUtils.isEmpty(lastTimestampForTable)) { +lastRefreshed = Long.parseLong(lastTimestampForTable); +} +if (lastRefreshed == -1 || (refreshInterval > 0 && currentTime >= (lastRefreshed + + refreshInterval))) { +stateMapProperties.remove(lastTimestampForTable); +} else { +refreshTable = false; +} +} catch (final NumberFormatException nfe) { +getLogger().error( + "Failed to r
[nifi] branch master updated: NIFI-6144 - Enable EL on listening port in HandleHttpRequest
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new ead6a3b NIFI-6144 - Enable EL on listening port in HandleHttpRequest ead6a3b is described below commit ead6a3b842422ca4dd9a34482e162ca80a0a55c5 Author: Pierre Villard AuthorDate: Sun Mar 24 12:21:21 2019 +0100 NIFI-6144 - Enable EL on listening port in HandleHttpRequest This closes #3386. Signed-off-by: Koji Kawamura --- .../java/org/apache/nifi/processors/standard/HandleHttpRequest.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/HandleHttpRequest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java index d03774e..fdd4d3d 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java @@ -153,7 +153,7 @@ public class HandleHttpRequest extends AbstractProcessor { .description("The Port to listen on for incoming HTTP requests") .required(true) .addValidator(StandardValidators.createLongValidator(0L, 65535L, true)) -.expressionLanguageSupported(ExpressionLanguageScope.NONE) + .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .defaultValue("80") .build(); public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder() @@ -326,7 +326,7 @@ public class HandleHttpRequest extends AbstractProcessor { } this.containerQueue = new LinkedBlockingQueue<>(context.getProperty(CONTAINER_QUEUE_SIZE).asInteger()); final String host = context.getProperty(HOSTNAME).getValue(); -final int port = context.getProperty(PORT).asInteger(); +final int port = context.getProperty(PORT).evaluateAttributeExpressions().asInteger(); final SSLContextService sslService = context.getProperty(SSL_CONTEXT).asControllerService(SSLContextService.class); final HttpContextMap httpContextMap = context.getProperty(HTTP_CONTEXT_MAP).asControllerService(HttpContextMap.class); final long requestTimeout = httpContextMap.getRequestTimeout(TimeUnit.MILLISECONDS);
[nifi] branch master updated: NIFI-6139: Add fragment attributes to PartitionRecord
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new ce09b93 NIFI-6139: Add fragment attributes to PartitionRecord ce09b93 is described below commit ce09b93ef1d104d65968054b97c2c9b2c5eaef3e Author: Matthew Burgess AuthorDate: Thu Mar 21 14:53:07 2019 -0400 NIFI-6139: Add fragment attributes to PartitionRecord This closes #3382. Signed-off-by: Koji Kawamura --- .../apache/nifi/processors/standard/PartitionRecord.java | 14 ++ .../nifi/processors/standard/TestPartitionRecord.java | 4 2 files changed, 18 insertions(+) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java index 43e1e4b..16209c8 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PartitionRecord.java @@ -29,6 +29,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -48,6 +49,7 @@ import org.apache.nifi.components.ValidationResult; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.flowfile.attributes.FragmentAttributes; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; @@ -84,6 +86,11 @@ import org.apache.nifi.serialization.record.util.DataTypeUtils; @WritesAttributes({ @WritesAttribute(attribute="record.count", description="The number of records in an outgoing FlowFile"), @WritesAttribute(attribute="mime.type", description="The MIME Type that the configured Record Writer indicates is appropriate"), +@WritesAttribute(attribute = "fragment.identifier", description = "All partitioned FlowFiles produced from the same parent FlowFile will have the same randomly " ++ "generated UUID added for this attribute"), +@WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"), +@WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile"), +@WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile"), @WritesAttribute(attribute="", description = "For each dynamic property that is added, an attribute may be added to the FlowFile. See the description for Dynamic Properties for more information.") }) @@ -232,6 +239,8 @@ public class PartitionRecord extends AbstractProcessor { } // For each RecordSetWriter, finish the record set and close the writer. +int fragmentIndex = 0; +final String fragmentId = UUID.randomUUID().toString(); for (final Map.Entry entry : writerMap.entrySet()) { final RecordValueMap valueMap = entry.getKey(); final RecordSetWriter writer = entry.getValue(); @@ -244,11 +253,16 @@ public class PartitionRecord extends AbstractProcessor { attributes.putAll(writeResult.getAttributes()); attributes.put("record.count", String.valueOf(writeResult.getRecordCount())); attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType()); +attributes.put(FragmentAttributes.FRAGMENT_INDEX.key(), String.valueOf(fragmentIndex)); +attributes.put(FragmentAttributes.FRAGMENT_ID.key(), fragmentId); +attributes.put(FragmentAttributes.FRAGMENT_COUNT.key(), String.valueOf(writerMap.size())); + attributes.put(FragmentAttributes.SEGMENT_ORIGINAL_FILENAME.key(), flowFile.getAttribute(CoreAttributes.FILENAME.key())); FlowFile childFlowFile = valueMap.getFlowFile(); childFlowFile = session.putAllAttributes(childFlowFile, attributes); session.adjustCounter("Record Processed", writeResult.getRecordCount(), false); +fragmentI
[nifi] 02/02: NIFI-5918 Fix issue with MergeRecord when DefragmentStrategy is on
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit cd3567873be7586aa285bd6b27e928e320299bee Author: Koji Kawamura AuthorDate: Wed Mar 20 12:16:16 2019 +0900 NIFI-5918 Fix issue with MergeRecord when DefragmentStrategy is on Added an unit test representing the fixed issue. And updated existing testDefragment test to illustrate the remaining FlowFiles those did not meet the threshold. --- .../nifi/processors/standard/TestMergeRecord.java | 63 -- 1 file changed, 59 insertions(+), 4 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeRecord.java index c54bf2a..3540b04 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeRecord.java @@ -158,27 +158,39 @@ public class TestMergeRecord { final Map attr1 = new HashMap<>(); attr1.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); attr1.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "1"); +attr1.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "0"); final Map attr2 = new HashMap<>(); attr2.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); attr2.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "1"); +attr2.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "1"); final Map attr3 = new HashMap<>(); attr3.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); attr3.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "2"); +attr3.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "0"); final Map attr4 = new HashMap<>(); attr4.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); -attr4.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "2"); +attr4.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "3"); +attr4.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "0"); + +final Map attr5 = new HashMap<>(); +attr5.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); +attr5.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "3"); +attr5.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "1"); runner.enqueue("Name, Age\nJohn, 35", attr1); runner.enqueue("Name, Age\nJane, 34", attr2); -runner.enqueue("Name, Age\nJake, 3", attr3); -runner.enqueue("Name, Age\nJan, 2", attr4); +runner.enqueue("Name, Age\nJay, 24", attr3); + +runner.enqueue("Name, Age\nJake, 3", attr4); +runner.enqueue("Name, Age\nJan, 2", attr5); -runner.run(4); +runner.run(1); +assertEquals("Fragment id=2 should remain in the incoming connection", 1, runner.getQueueSize().getObjectCount()); runner.assertTransferCount(MergeRecord.REL_MERGED, 2); runner.assertTransferCount(MergeRecord.REL_ORIGINAL, 4); @@ -196,6 +208,49 @@ public class TestMergeRecord { @Test +public void testDefragmentWithMultipleRecords() { +runner.setProperty(MergeRecord.MERGE_STRATEGY, MergeRecord.MERGE_STRATEGY_DEFRAGMENT); + +final Map attr1 = new HashMap<>(); +attr1.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); +attr1.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "1"); +attr1.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "0"); +attr1.put("record.count", "2"); + +final Map attr2 = new HashMap<>(); +attr2.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); +attr2.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "1"); +attr2.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "1"); +attr2.put("record.count", "2"); + +final Map attr3 = new HashMap<>(); +attr3.put(MergeRecord.FRAGMENT_COUNT_ATTRIBUTE, "2"); +attr3.put(MergeRecord.FRAGMENT_ID_ATTRIBUTE, "2"); +attr3.put(MergeRecord.FRAGMENT_INDEX_ATTRIBUTE, "0"); +attr3.put("record.count", "2"); + +runner.enqueue("Name, Age\nJohn, 35\nJane, 34", attr1); + +runner.enqueue("Name, Age\nJake, 3\nJan, 2", attr2); + +runner.enqueue("Name, Age\nJay, 24\nJade, 28", attr3); + +runner.run(1); + +assertEquals("Fr
[nifi] 01/02: NIFI-5918 Fix issue with MergeRecord when DefragmentStrategy is on
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit b97fbd2c89caafec4bde485c6118a6daa087fcb2 Author: Andres Garagiola AuthorDate: Fri Feb 22 17:15:59 2019 -0300 NIFI-5918 Fix issue with MergeRecord when DefragmentStrategy is on This closes #3334. Signed-off-by: Andres Garagiola Signed-off-by: Koji Kawamura --- .../nifi/processors/standard/merge/RecordBin.java | 55 +- .../standard/merge/RecordBinManager.java | 10 ++-- .../standard/merge/RecordBinThresholds.java| 10 ++-- 3 files changed, 31 insertions(+), 44 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java index d15ba0f..a96e119 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/merge/RecordBin.java @@ -63,6 +63,7 @@ public class RecordBin { private RecordSetWriter recordWriter; private ByteCountingOutputStream out; private int recordCount = 0; +private int fragmentCount = 0; private volatile boolean complete = false; private static final AtomicLong idGenerator = new AtomicLong(0L); @@ -114,7 +115,7 @@ public class RecordBin { } boolean flowFileMigrated = false; - +this.fragmentCount++; try { if (isComplete()) { logger.debug("RecordBin.offer for id={} returning false because {} is complete", new Object[] {flowFile.getId(), this}); @@ -202,26 +203,7 @@ public class RecordBin { return false; } -int maxRecords; -final Optional recordCountAttribute = thresholds.getRecordCountAttribute(); -if (recordCountAttribute.isPresent()) { -final Optional recordCountValue = flowFiles.stream() -.filter(ff -> ff.getAttribute(recordCountAttribute.get()) != null) -.map(ff -> ff.getAttribute(recordCountAttribute.get())) -.findFirst(); - -if (!recordCountValue.isPresent()) { -return false; -} - -try { -maxRecords = Integer.parseInt(recordCountValue.get()); -} catch (final NumberFormatException e) { -maxRecords = 1; -} -} else { -maxRecords = thresholds.getMaxRecords(); -} +int maxRecords = thresholds.getMaxRecords(); if (recordCount >= maxRecords) { return true; @@ -231,6 +213,22 @@ public class RecordBin { return true; } +Optional fragmentCountAttribute = thresholds.getFragmentCountAttribute(); +if(fragmentCountAttribute != null && fragmentCountAttribute.isPresent()) { +final Optional fragmentCountValue = flowFiles.stream() +.filter(ff -> ff.getAttribute(fragmentCountAttribute.get()) != null) +.map(ff -> ff.getAttribute(fragmentCountAttribute.get())) +.findFirst(); +if (fragmentCountValue.isPresent()) { +try { +int expectedFragments = Integer.parseInt(fragmentCountValue.get()); +if (this.fragmentCount == expectedFragments) +return true; +} catch (NumberFormatException nfe) { +this.logger.error(nfe.getMessage(), nfe); +} +} +} return false; } finally { readLock.unlock(); @@ -243,18 +241,7 @@ public class RecordBin { return currentCount; } -int requiredCount; -final Optional recordCountAttribute = thresholds.getRecordCountAttribute(); -if (recordCountAttribute.isPresent()) { -final String recordCountValue = flowFiles.get(0).getAttribute(recordCountAttribute.get()); -try { -requiredCount = Integer.parseInt(recordCountValue); -} catch (final NumberFormatException e) { -requiredCount = 1; -} -} else { -requiredCount = thresholds.getMinRecords(); -} +int requiredCount = thresholds.getMinRecords(); this.requiredRecordCount = requiredCount; return requiredCount; @@ -347,7
[nifi] branch master updated (9d21a10 -> cd35678)
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from 9d21a10 NIFI-6120 Move nifi.registry.version property to root pom so it can be shared between nifi-framework and nifi-toolkit new b97fbd2 NIFI-5918 Fix issue with MergeRecord when DefragmentStrategy is on new cd35678 NIFI-5918 Fix issue with MergeRecord when DefragmentStrategy is on The 2 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: .../nifi/processors/standard/merge/RecordBin.java | 55 --- .../standard/merge/RecordBinManager.java | 10 ++-- .../standard/merge/RecordBinThresholds.java| 10 ++-- .../nifi/processors/standard/TestMergeRecord.java | 63 -- 4 files changed, 90 insertions(+), 48 deletions(-)
[nifi] branch master updated: NIFI-6120 Move nifi.registry.version property to root pom so it can be shared between nifi-framework and nifi-toolkit
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 9d21a10 NIFI-6120 Move nifi.registry.version property to root pom so it can be shared between nifi-framework and nifi-toolkit 9d21a10 is described below commit 9d21a106ad6674ab4f12828b9dd007f28e5808f2 Author: Bryan Bende AuthorDate: Wed Mar 13 14:00:41 2019 -0400 NIFI-6120 Move nifi.registry.version property to root pom so it can be shared between nifi-framework and nifi-toolkit This closes #3369. Signed-off-by: Koji Kawamura --- nifi-nar-bundles/nifi-framework-bundle/pom.xml | 1 - nifi-toolkit/nifi-toolkit-cli/pom.xml | 2 +- pom.xml| 1 + 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml index 5b2c6dc..1ba5b8d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml @@ -23,7 +23,6 @@ pom NiFi: Framework Bundle -0.3.0 2.26 4.3.19.RELEASE 4.2.8.RELEASE diff --git a/nifi-toolkit/nifi-toolkit-cli/pom.xml b/nifi-toolkit/nifi-toolkit-cli/pom.xml index 25e0aaa..3df8252 100644 --- a/nifi-toolkit/nifi-toolkit-cli/pom.xml +++ b/nifi-toolkit/nifi-toolkit-cli/pom.xml @@ -64,7 +64,7 @@ org.apache.nifi.registry nifi-registry-client -0.2.0 +${nifi.registry.version} com.fasterxml.jackson.core diff --git a/pom.xml b/pom.xml index 8037976..72dabd7 100644 --- a/pom.xml +++ b/pom.xml @@ -96,6 +96,7 @@ 1.0.0 9.4.11.v20180605 2.9.8 +0.3.0
[nifi] branch master updated: NIFI-6122: PutS3Object supports national characters in filenames.
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 77b84ed NIFI-6122: PutS3Object supports national characters in filenames. 77b84ed is described below commit 77b84edf5bd3eca03d7b2ef1d17e4cf6e2f5e15c Author: Peter Turcsanyi AuthorDate: Thu Mar 7 23:18:20 2019 +0100 NIFI-6122: PutS3Object supports national characters in filenames. This closes #3373. Signed-off-by: Koji Kawamura --- .../apache/nifi/processors/aws/s3/PutS3Object.java | 3 +- .../nifi/processors/aws/s3/ITPutS3Object.java | 17 + .../nifi/processors/aws/s3/TestPutS3Object.java| 76 +- 3 files changed, 65 insertions(+), 31 deletions(-) diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java index dfbcbda..d331d15 100644 --- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java +++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java @@ -23,6 +23,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.Serializable; +import java.net.URLEncoder; import java.nio.file.Files; import java.text.DateFormat; import java.text.SimpleDateFormat; @@ -445,7 +446,7 @@ public class PutS3Object extends AbstractS3Processor { public void process(final InputStream rawIn) throws IOException { try (final InputStream in = new BufferedInputStream(rawIn)) { final ObjectMetadata objectMetadata = new ObjectMetadata(); - objectMetadata.setContentDisposition(ff.getAttribute(CoreAttributes.FILENAME.key())); + objectMetadata.setContentDisposition(URLEncoder.encode(ff.getAttribute(CoreAttributes.FILENAME.key()), "UTF-8")); objectMetadata.setContentLength(ff.getSize()); final String contentType = context.getProperty(CONTENT_TYPE) diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java index 2db139a..5cb1a97 100644 --- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java +++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java @@ -119,6 +119,23 @@ public class ITPutS3Object extends AbstractS3IT { } } +@Test +public void testSimplePutFilenameWithNationalCharacters() throws IOException { +final TestRunner runner = TestRunners.newTestRunner(new PutS3Object()); + +runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE); +runner.setProperty(PutS3Object.REGION, REGION); +runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME); + +final Map attrs = new HashMap<>(); +attrs.put("filename", "Iñtërnâtiônàližætiøn.txt"); +runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs); + +runner.run(1); + +runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1); +} + private void testPutThenFetch(String sseAlgorithm) throws IOException { // Put diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestPutS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestPutS3Object.java index 2c3db81..67ebae7 100644 --- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestPutS3Object.java +++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestPutS3Object.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.processors.aws.s3; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; import java.util.Date; import java.util.HashMap; import java.util.List; @@ -53,27 +55,28 @@ import static org.junit.Assert.assertTrue; public class TestPutS3Object { -private TestRunner runner = null; -private PutS3Object mockPutS3Object = null; -private AmazonS3Client actualS3Client = null; -private AmazonS3Client mockS3Client = null; + +private TestRunner runner; +private PutS3Object putS3Object; +private AmazonS
[nifi] branch master updated: NIFI-6127: Fixed NPE in match() and find() EL functions when attribute doesn't exist
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 06f41ac NIFI-6127: Fixed NPE in match() and find() EL functions when attribute doesn't exist 06f41ac is described below commit 06f41ac6f8645fc2e37be2529200e89177f48197 Author: Matthew Burgess AuthorDate: Tue Mar 19 14:03:57 2019 -0400 NIFI-6127: Fixed NPE in match() and find() EL functions when attribute doesn't exist This closes #3378. Signed-off-by: Koji Kawamura --- .../expression/language/evaluation/functions/FindEvaluator.java | 6 +- .../expression/language/evaluation/functions/MatchesEvaluator.java | 6 +- .../org/apache/nifi/attribute/expression/language/TestQuery.java| 6 ++ 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java index 2a3f1ae..fa2aaf5 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java @@ -53,7 +53,11 @@ public class FindEvaluator extends BooleanEvaluator { } final Pattern pattern; if (compiledPattern == null) { -pattern = Pattern.compile(search.evaluate(attributes).getValue()); +String expression = search.evaluate(attributes).getValue(); +if (expression == null) { +return new BooleanQueryResult(false); +} +pattern = Pattern.compile(expression); } else { pattern = compiledPattern; } diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java index 6f017ea..d238938 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java @@ -53,7 +53,11 @@ public class MatchesEvaluator extends BooleanEvaluator { } final Pattern pattern; if (compiledPattern == null) { -pattern = Pattern.compile(search.evaluate(attributes).getValue()); +String expression = search.evaluate(attributes).getValue(); +if (expression == null) { +return new BooleanQueryResult(false); +} +pattern = Pattern.compile(expression); } else { pattern = compiledPattern; } 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 67e5b51..1c679a4 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 @@ -1324,6 +1324,9 @@ public class TestQuery { assertEquals("false", secondEvaluation); verifyEquals("${dotted:matches('abc\\.xyz')}", attributes, true); + +// Test for matches(null) +assertEquals("false", Query.evaluateExpressions("${abc:matches(${not.here})}", attributes, null)); } @Test @@ -1344,6 +1347,9 @@ public class TestQuery { assertEquals("false", secondEvaluation); verifyEquals("${dotted:find('\\.')}", attributes, true); + +// Test for find(null) +assertEquals("false", Query.evaluateExpressions("${abc:find(${not.here})}", attributes, null)); } @Test
[nifi] 02/02: NIFI-6082: Refactor the way to handle fields nullable
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit 4db5446c878a9be1d621429686f2835dc642d550 Author: Koji Kawamura AuthorDate: Wed Mar 13 16:50:01 2019 +0900 NIFI-6082: Refactor the way to handle fields nullable - Make enriched fields nullable at LookupRecord. - Removed unnecessary AvroConversionOptions and reader schema creation, because ResultSetRecordSet can generate NiFi Record Schema from RS directly. No Avro schema is needed to do that. --- .../serialization/record/ResultSetRecordSet.java | 18 ++ .../nifi/processors/standard/LookupRecord.java | 11 -- .../nifi/processors/standard/TestLookupRecord.java | 40 ++ .../lookup/db/DatabaseRecordLookupService.java | 18 +- 4 files changed, 53 insertions(+), 34 deletions(-) diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java index fc3d60f..ee47c63 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java @@ -55,21 +55,9 @@ public class ResultSetRecordSet implements RecordSet, Closeable { private static final String FLOAT_CLASS_NAME = Float.class.getName(); public ResultSetRecordSet(final ResultSet rs, final RecordSchema readerSchema) throws SQLException { -this(rs, readerSchema, false); -} - -/** - * Constructs a ResultSetRecordSet with a given ResultSet and RecordSchema - * - * @param rs The underlying ResultSet for this RecordSet - * @param readerSchema The schema to which this RecordSet adheres - * @param allFieldsNullable Whether to override the database column's "nullable" metadata. If true then all fields in the RecordSet are nullable. - * @throws SQLException if an error occurs while creating the schema or reading the result set's metadata - */ -public ResultSetRecordSet(final ResultSet rs, final RecordSchema readerSchema, boolean allFieldsNullable) throws SQLException { this.rs = rs; moreRows = rs.next(); -this.schema = createSchema(rs, readerSchema, allFieldsNullable); +this.schema = createSchema(rs, readerSchema); rsColumnNames = new HashSet<>(); final ResultSetMetaData metadata = rs.getMetaData(); @@ -152,7 +140,7 @@ public class ResultSetRecordSet implements RecordSet, Closeable { return value; } -private static RecordSchema createSchema(final ResultSet rs, final RecordSchema readerSchema, boolean allFieldsNullable) throws SQLException { +private static RecordSchema createSchema(final ResultSet rs, final RecordSchema readerSchema) throws SQLException { final ResultSetMetaData metadata = rs.getMetaData(); final int numCols = metadata.getColumnCount(); final List fields = new ArrayList<>(numCols); @@ -166,7 +154,7 @@ public class ResultSetRecordSet implements RecordSet, Closeable { final int nullableFlag = metadata.isNullable(column); final boolean nullable; -if (nullableFlag == ResultSetMetaData.columnNoNulls && !allFieldsNullable) { +if (nullableFlag == ResultSetMetaData.columnNoNulls) { nullable = false; } else { nullable = true; 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 96a8d3e..23d1325 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 @@ -316,7 +316,7 @@ public class LookupRecord extends AbstractRouteRecord recordFieldOption = lookupRecord.getSchema().getField(fieldName); if (recordFieldOption.isPresent()) { - destinationRecord.setValue(recordFieldOption.get(), value); +// Even if the looked up field is not nullable, if the lookup key didn't match with any record, +// and matched/unmatched records are written to the same FlowFile routed to 'success' relationship, +// then enriched fields should be nullable to support unmatched records whose enriched fields will be null. +
[nifi] 01/02: NIFI-6082: Added DatabaseRecordLookupService, refactored common DB utils
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit ca76fe178cfae8890b347081260cd59a62321219 Author: Matthew Burgess AuthorDate: Wed Feb 27 16:17:46 2019 -0500 NIFI-6082: Added DatabaseRecordLookupService, refactored common DB utils NIFI-6082: Added SimpleDatabaseLookupService NIFI-6082: Merged Koji's improvements, incorporated review comments This closes #3341. Signed-off-by: Koji Kawamura --- .../serialization/record/ResultSetRecordSet.java | 18 +- .../{ => nifi-database-test-utils}/pom.xml | 26 +-- .../apache/nifi/util/db/SimpleCommerceDataSet.java | 112 ++ .../nifi-database-utils/pom.xml| 101 + .../java/org/apache/nifi/util/db}/AvroUtil.java| 2 +- .../java/org/apache/nifi/util/db}/JdbcCommon.java | 71 +-- .../apache/nifi/util/db}/JdbcCommonTestUtils.java | 2 +- .../org/apache/nifi/util/db}/TestJdbcCommon.java | 18 +- .../nifi/util/db}/TestJdbcCommonConvertToAvro.java | 8 +- .../apache/nifi/util/db}/TestJdbcHugeStream.java | 2 +- .../apache/nifi/util/db}/TestJdbcTypesDerby.java | 2 +- .../org/apache/nifi/util/db}/TestJdbcTypesH2.java | 2 +- nifi-nar-bundles/nifi-extension-utils/pom.xml | 2 + .../nifi-standard-processors/pom.xml | 11 + .../processors/standard/AbstractExecuteSQL.java| 2 +- .../standard/AbstractQueryDatabaseTable.java | 2 +- .../nifi/processors/standard/ExecuteSQL.java | 12 +- .../nifi/processors/standard/ExecuteSQLRecord.java | 5 +- .../nifi/processors/standard/LookupRecord.java | 5 +- .../apache/nifi/processors/standard/PutSQL.java| 2 +- .../processors/standard/QueryDatabaseTable.java| 10 +- .../standard/QueryDatabaseTableRecord.java | 4 +- .../standard/sql/DefaultAvroSqlWriter.java | 12 +- .../processors/standard/sql/RecordSqlWriter.java | 14 +- .../nifi/processors/standard/sql/SqlWriter.java| 6 +- .../processors/standard/util/JdbcProperties.java | 81 .../nifi/processors/standard/TestExecuteSQL.java | 6 +- .../processors/standard/TestExecuteSQLRecord.java | 4 +- .../nifi-lookup-services/pom.xml | 29 ++- .../lookup/db/AbstractDatabaseLookupService.java | 104 ++ .../lookup/db/DatabaseRecordLookupService.java | 206 ++ .../lookup/db/SimpleDatabaseLookupService.java | 174 .../org.apache.nifi.controller.ControllerService | 2 + .../db/TestDatabaseRecordLookupService.groovy | 229 + .../db/TestSimpleDatabaseLookupService.groovy | 184 + 35 files changed, 1322 insertions(+), 148 deletions(-) diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java index ee47c63..fc3d60f 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java @@ -55,9 +55,21 @@ public class ResultSetRecordSet implements RecordSet, Closeable { private static final String FLOAT_CLASS_NAME = Float.class.getName(); public ResultSetRecordSet(final ResultSet rs, final RecordSchema readerSchema) throws SQLException { +this(rs, readerSchema, false); +} + +/** + * Constructs a ResultSetRecordSet with a given ResultSet and RecordSchema + * + * @param rs The underlying ResultSet for this RecordSet + * @param readerSchema The schema to which this RecordSet adheres + * @param allFieldsNullable Whether to override the database column's "nullable" metadata. If true then all fields in the RecordSet are nullable. + * @throws SQLException if an error occurs while creating the schema or reading the result set's metadata + */ +public ResultSetRecordSet(final ResultSet rs, final RecordSchema readerSchema, boolean allFieldsNullable) throws SQLException { this.rs = rs; moreRows = rs.next(); -this.schema = createSchema(rs, readerSchema); +this.schema = createSchema(rs, readerSchema, allFieldsNullable); rsColumnNames = new HashSet<>(); final ResultSetMetaData metadata = rs.getMetaData(); @@ -140,7 +152,7 @@ public class ResultSetRecordSet implements RecordSet, Closeable { return value; } -private static RecordSchema createSchema(final ResultSet rs, final RecordSchema readerSchema) throws SQLException { +private static RecordSchema createSchema(final ResultSet rs, final RecordSchema readerSchema, boolean allFieldsNullable) throws SQLException { final ResultS
[nifi] branch master updated (0e10e41 -> 4db5446)
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from 0e10e41 NIFI-4358 This closes #3363. cassandra connection enable compression at resquest and response new ca76fe1 NIFI-6082: Added DatabaseRecordLookupService, refactored common DB utils new 4db5446 NIFI-6082: Refactor the way to handle fields nullable The 2 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: .../nifi-database-test-utils}/pom.xml | 16 +- .../apache/nifi/util/db/SimpleCommerceDataSet.java | 112 ++ .../pom.xml| 44 +++- .../java/org/apache/nifi/util/db}/AvroUtil.java| 2 +- .../java/org/apache/nifi/util/db}/JdbcCommon.java | 71 +-- .../apache/nifi/util/db}/JdbcCommonTestUtils.java | 2 +- .../org/apache/nifi/util/db}/TestJdbcCommon.java | 18 +- .../nifi/util/db}/TestJdbcCommonConvertToAvro.java | 8 +- .../apache/nifi/util/db}/TestJdbcHugeStream.java | 2 +- .../apache/nifi/util/db}/TestJdbcTypesDerby.java | 2 +- .../org/apache/nifi/util/db}/TestJdbcTypesH2.java | 2 +- nifi-nar-bundles/nifi-extension-utils/pom.xml | 2 + .../nifi-standard-processors/pom.xml | 11 + .../processors/standard/AbstractExecuteSQL.java| 2 +- .../standard/AbstractQueryDatabaseTable.java | 2 +- .../nifi/processors/standard/ExecuteSQL.java | 12 +- .../nifi/processors/standard/ExecuteSQLRecord.java | 5 +- .../nifi/processors/standard/LookupRecord.java | 16 +- .../apache/nifi/processors/standard/PutSQL.java| 2 +- .../processors/standard/QueryDatabaseTable.java| 10 +- .../standard/QueryDatabaseTableRecord.java | 4 +- .../standard/sql/DefaultAvroSqlWriter.java | 12 +- .../processors/standard/sql/RecordSqlWriter.java | 14 +- .../nifi/processors/standard/sql/SqlWriter.java| 6 +- .../processors/standard/util/JdbcProperties.java | 81 .../nifi/processors/standard/TestExecuteSQL.java | 6 +- .../processors/standard/TestExecuteSQLRecord.java | 4 +- .../nifi/processors/standard/TestLookupRecord.java | 40 .../nifi-lookup-services/pom.xml | 29 ++- .../lookup/db/AbstractDatabaseLookupService.java | 104 ++ .../lookup/db/DatabaseRecordLookupService.java | 190 + .../lookup/db/SimpleDatabaseLookupService.java | 174 .../org.apache.nifi.controller.ControllerService | 2 + .../db/TestDatabaseRecordLookupService.groovy | 229 + .../db/TestSimpleDatabaseLookupService.groovy | 184 + 35 files changed, 1274 insertions(+), 146 deletions(-) copy nifi-nar-bundles/{nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access => nifi-extension-utils/nifi-database-test-utils}/pom.xml (71%) create mode 100644 nifi-nar-bundles/nifi-extension-utils/nifi-database-test-utils/src/main/java/org/apache/nifi/util/db/SimpleCommerceDataSet.java copy nifi-nar-bundles/nifi-extension-utils/{nifi-record-utils/nifi-avro-record-utils => nifi-database-utils}/pom.xml (73%) mode change 100755 => 100644 rename nifi-nar-bundles/{nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util => nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db}/AvroUtil.java (97%) rename nifi-nar-bundles/{nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util => nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db}/JdbcCommon.java (90%) rename nifi-nar-bundles/{nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util => nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db}/JdbcCommonTestUtils.java (97%) rename nifi-nar-bundles/{nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util => nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db}/TestJdbcCommon.java (97%) rename nifi-nar-bundles/{nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util => nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db}/TestJdbcCommonConvertToAvro.java (92%) rename nifi-nar-bundles/{nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util => nifi-extension-utils/nifi-database-utils/src/test/java/org/apache/nifi/util/db}/TestJdbcHugeStream.java (99%) rename nifi-nar-bundles/{nifi-standard-bu
[nifi] branch master updated: NIFI-3685 Azure EventHub processor tests fail
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new d53cefa NIFI-3685 Azure EventHub processor tests fail d53cefa is described below commit d53cefa2a75d47b5e75c6fb55e86bada759cce79 Author: Ferenc Szabó AuthorDate: Mon Mar 4 14:30:56 2019 +0100 NIFI-3685 Azure EventHub processor tests fail Moved test classes into the proper folder. Fixed outdated mocking and added assertions that actually tests the content. This closes #3346. Signed-off-by: Koji Kawamura --- .../azure/eventhub/GetAzureEventHubTest.java | 75 +- .../azure/eventhub/PutAzureEventHubTest.java | 4 +- 2 files changed, 62 insertions(+), 17 deletions(-) diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java similarity index 69% rename from nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java rename to nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java index 951384a..a3b80bf 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHubTest.java @@ -17,21 +17,28 @@ package org.apache.nifi.processors.azure.eventhub; import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventData.SystemProperties; import com.microsoft.azure.eventhubs.PartitionReceiver; import com.microsoft.azure.servicebus.ServiceBusException; +import com.microsoft.azure.servicebus.amqp.AmqpConstants; + +import java.io.IOException; +import java.time.Clock; +import java.time.Instant; +import java.time.ZoneId; +import java.util.Date; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.concurrent.ExecutionException; + import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.Before; import org.junit.Test; -import org.powermock.reflect.Whitebox; - -import java.io.IOException; -import java.time.Instant; -import java.time.temporal.ChronoUnit; -import java.util.LinkedList; -import java.util.concurrent.ExecutionException; +import org.mockito.internal.util.reflection.Whitebox; public class GetAzureEventHubTest { @@ -40,6 +47,10 @@ public class GetAzureEventHubTest { private static final String eventHubName = "get-test"; private static final String sasKeyName = "bogus-policy"; private static final String sasKey = "9rHmHqxoOVWOb8wS09dvqXYxnNiLqxNMCbmt6qMaQyU!"; +private static final Date ENQUEUED_TIME_VALUE = Date.from(Clock.fixed(Instant.now(), ZoneId.systemDefault()).instant()); +public static final long SEQUENCE_NUMBER_VALUE = 13L; +public static final String OFFSET_VALUE = "100"; +public static final String PARTITION_KEY_VALUE = "0"; private TestRunner testRunner; private MockGetAzureEventHub processor; @@ -69,13 +80,12 @@ public class GetAzureEventHubTest { testRunner.setProperty(GetAzureEventHub.RECEIVER_FETCH_TIMEOUT,"1"); testRunner.assertValid(); } - + @Test public void verifyRelationships(){ - assert(1 == processor.getRelationships().size()); - } + @Test public void testNoPartitions(){ MockGetAzureEventHubNoPartitions mockProcessor = new MockGetAzureEventHubNoPartitions(); @@ -85,6 +95,7 @@ public class GetAzureEventHubTest { testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 0); testRunner.clearTransferState(); } + @Test public void testNullRecieve(){ setUpStandardTestConfig(); @@ -93,6 +104,7 @@ public class GetAzureEventHubTest { testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 0); testRunner.clearTransferState(); } + @Test(expected = AssertionError.class) public void testThrowGetReceiver(){ setUpStandardTestConfig(); @@ -101,12 +113,37 @@ public class GetAzureEventHubTest { testRunner.assertAllFlowFilesTransferred(GetAzureEventHub.REL_SUCCESS, 0); testRunner.clearTransferState(); } +
[nifi] branch master updated: NIFI-5841 Fix memory leak of PutHive3Streaming.
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new e6e4175 NIFI-5841 Fix memory leak of PutHive3Streaming. e6e4175 is described below commit e6e4175d71d23157c152e7337e97fc45533b3869 Author: kei miyauchi AuthorDate: Tue Jan 8 09:38:46 2019 +0900 NIFI-5841 Fix memory leak of PutHive3Streaming. This closes #3249. Signed-off-by: Koji Kawamura --- .../main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java index 810471b..81916c6 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java @@ -412,9 +412,6 @@ public class PutHive3Streaming extends AbstractProcessor { } hiveStreamingConnection = makeStreamingConnection(options, reader); -// Add shutdown handler with higher priority than FileSystem shutdown hook so that streaming connection gets closed first before -// filesystem close (to avoid ClosedChannelException) - ShutdownHookManager.addShutdownHook(hiveStreamingConnection::close, FileSystem.SHUTDOWN_HOOK_PRIORITY + 1); // Write records to Hive streaming, then commit and close hiveStreamingConnection.beginTransaction();
[nifi] branch master updated: NIFI-5920: Tagging an object in S3
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new a8e59e5 NIFI-5920: Tagging an object in S3 a8e59e5 is described below commit a8e59e52af5144c6238230bd792d01ca6daafadf Author: Stephen Goodman AuthorDate: Sat Dec 29 19:10:38 2018 -0500 NIFI-5920: Tagging an object in S3 Unit tests and functionality for tagging an object in S3. Set FlowFile attributes directly from tags retrieved from S3. Add guard clauses to ensure evaluated properties are not blank. This closes #3239. Signed-off-by: Koji Kawamura --- .../apache/nifi/processors/aws/s3/TagS3Object.java | 201 + .../services/org.apache.nifi.processor.Processor | 1 + .../nifi/processors/aws/s3/ITTagS3Object.java | 150 ++ .../nifi/processors/aws/s3/TestTagS3Object.java| 319 + 4 files changed, 671 insertions(+) diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/TagS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/TagS3Object.java new file mode 100644 index 000..6c9d72a --- /dev/null +++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/TagS3Object.java @@ -0,0 +1,201 @@ +/* + * 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.processors.aws.s3; + +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.GetObjectTaggingRequest; +import com.amazonaws.services.s3.model.GetObjectTaggingResult; +import com.amazonaws.services.s3.model.ObjectTagging; +import com.amazonaws.services.s3.model.SetObjectTaggingRequest; +import com.amazonaws.services.s3.model.Tag; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +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.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.util.StringUtils; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + + +@SupportsBatching +@WritesAttributes({ +@WritesAttribute(attribute = "s3.tag.___", description = "The tags associated with the S3 object will be " + +"written as part of the FlowFile attributes")}) +@SeeAlso({PutS3Object.class, FetchS3Object.class, ListS3.class}) +@Tags({"Amazon", "S3", "AWS", "Archive", "Tag"}) +@InputRequirement(Requirement.INPUT_REQUIRED) +@CapabilityDescription("Sets tags on a FlowFile within an Amazon S3 Bucket. " + +"If attempting to tag a file that does not exist, FlowFile is routed to success.") +public class TagS3Object extends AbstractS3Processor { + +public static final PropertyDescriptor TAG_KEY = new PropertyDescriptor.Builder() +.name("tag-key") +.displayName("Tag Key") +.description("The key of the tag that will be set on the S3 Object") +.addValidator(new StandardValidat
[nifi] branch master updated: NIFI-5892 Wait timestamp lingers, potentially messing up downstream wait-notify pairs Clear the wait timestamp when transferring to failur or success
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 1330b92 NIFI-5892 Wait timestamp lingers, potentially messing up downstream wait-notify pairs Clear the wait timestamp when transferring to failur or success 1330b92 is described below commit 1330b92cfa01f9dd04bade7a05a047ad6f080c4b Author: Otto Fowler AuthorDate: Fri Dec 21 09:15:24 2018 -0500 NIFI-5892 Wait timestamp lingers, potentially messing up downstream wait-notify pairs Clear the wait timestamp when transferring to failur or success replace explicit attribute clear with function call, refactor and integrate into existing tests per review This closes #3233. Signed-off-by: Koji Kawamura --- .../org/apache/nifi/processors/standard/Wait.java | 23 +++--- .../apache/nifi/processors/standard/TestWait.java | 27 -- 2 files changed, 45 insertions(+), 5 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java index cfec15e..e297556 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java @@ -84,7 +84,8 @@ import static org.apache.nifi.processor.FlowFileFilter.FlowFileFilterResult.REJE ) @WritesAttributes({ @WritesAttribute(attribute = "wait.start.timestamp", description = "All FlowFiles will have an attribute 'wait.start.timestamp', which sets the " -+ "initial epoch timestamp when the file first entered this processor. This is used to determine the expiration time of the FlowFile."), ++ "initial epoch timestamp when the file first entered this processor. This is used to determine the expiration time of the FlowFile. " ++ "This attribute is not written when the FlowFile is transferred to failure or success"), @WritesAttribute(attribute = "wait.counter.", description = "If a signal exists when the processor runs, " + "each count value in the signal is copied.") }) @@ -314,6 +315,8 @@ public class Wait extends AbstractProcessor { final Consumer transferToFailure = flowFile -> { flowFile = session.penalize(flowFile); +// This flowFile is now failed, our tracking is done, clear the timer +flowFile = clearWaitState(session, flowFile); getFlowFilesFor.apply(REL_FAILURE).add(flowFile); }; @@ -328,9 +331,19 @@ public class Wait extends AbstractProcessor { relationship = Relationship.SELF; } } - +final Relationship finalRelationship = relationship; final List flowFilesWithSignalAttributes = routedFlowFiles.getValue().stream() -.map(f -> copySignalAttributes(session, f, signalRef.get(), originalSignalCounts, replaceOriginalAttributes)).collect(Collectors.toList()); +.map(f -> { +if (REL_SUCCESS.equals(finalRelationship)) { +// These flowFiles will be exiting the wait, clear the timer +f = clearWaitState(session, f); +} +return copySignalAttributes(session, f, signalRef.get(), +originalSignalCounts, +replaceOriginalAttributes); +}) +.collect(Collectors.toList()); + session.transfer(flowFilesWithSignalAttributes, relationship); }; @@ -470,6 +483,10 @@ public class Wait extends AbstractProcessor { } +private FlowFile clearWaitState(final ProcessSession session, final FlowFile flowFile) { +return session.removeAttribute(flowFile, WAIT_START_TIMESTAMP); +} + private FlowFile copySignalAttributes(final ProcessSession session, final FlowFile flowFile, final Signal signal, final Map originalCount, final boolean replaceOriginal) { if (signal == null) { return flowFile; diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestWait.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestWait.java index 5b5b6fc..2ccb2fe 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src
[nifi] branch master updated: NIFI-4892 - ValidateCSV: no doublequote escaping in invalid output
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 590fa20 NIFI-4892 - ValidateCSV: no doublequote escaping in invalid output 590fa20 is described below commit 590fa2063cd915123c5f46dac07890ff7607ffb0 Author: Pierre Villard AuthorDate: Tue Feb 20 10:36:31 2018 +0100 NIFI-4892 - ValidateCSV: no doublequote escaping in invalid output NIFI-4892 - ValidateCSV: no doublequote escaping in invalid output NIFI-5907 - unit test This closes #2481. Signed-off-by: Koji Kawamura --- .../nifi/processors/standard/ValidateCsv.java | 52 ++ .../nifi/processors/standard/TestValidateCsv.java | 50 - 2 files changed, 62 insertions(+), 40 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java index 9796822..bd6a0e7 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java @@ -466,24 +466,25 @@ public class ValidateCsv extends AbstractProcessor { session.read(flowFile, new InputStreamCallback() { @Override public void process(final InputStream in) throws IOException { -NifiCsvListReader listReader = null; -try { -listReader = new NifiCsvListReader(new InputStreamReader(in), csvPref); +try(final NifiCsvListReader listReader = new NifiCsvListReader(new InputStreamReader(in), csvPref)) { // handling of header if(header) { -List headerList = listReader.read(); + +// read header +listReader.read(); + if(!isWholeFFValidation) { invalidFF.set(session.append(invalidFF.get(), new OutputStreamCallback() { @Override public void process(OutputStream out) throws IOException { -out.write(print(headerList, csvPref, isFirstLineInvalid.get())); + out.write(print(listReader.getUntokenizedRow(), csvPref, true)); } })); validFF.set(session.append(validFF.get(), new OutputStreamCallback() { @Override public void process(OutputStream out) throws IOException { -out.write(print(headerList, csvPref, isFirstLineValid.get())); + out.write(print(listReader.getUntokenizedRow(), csvPref, true)); } })); isFirstLineValid.set(false); @@ -496,14 +497,14 @@ public class ValidateCsv extends AbstractProcessor { while (!stop) { try { -final List list = listReader.read(cellProcs); -stop = list == null; +// read next row and check if no more row +stop = listReader.read(cellProcs) == null; if(!isWholeFFValidation && !stop) { validFF.set(session.append(validFF.get(), new OutputStreamCallback() { @Override public void process(OutputStream out) throws IOException { -out.write(print(list, csvPref, isFirstLineValid.get())); + out.write(print(listReader.getUntokenizedRow(), csvPref, isFirstLineValid.get())); } })); okCount.set(okCount.get() + 1); @@ -524,7 +525,7 @@ public class ValidateCsv extends AbstractProcessor { invalidFF.set(session.append(invalidFF.get(), new OutputStreamCallback() { @Override public void process(OutputStream out) throws IOException { - out.write(print(e.getCsvContext().getRowSource(), csvPref, isFirstLineInv
[nifi] branch master updated: NIFI-3988: Add fragment attributes to SplitRecord
This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 7548d7c NIFI-3988: Add fragment attributes to SplitRecord 7548d7c is described below commit 7548d7c85914d9c64efeccce62d61de522170286 Author: Matthew Burgess AuthorDate: Thu Dec 13 14:02:14 2018 -0500 NIFI-3988: Add fragment attributes to SplitRecord This closes #3217. Signed-off-by: Koji Kawamura --- .../nifi/processors/standard/SplitRecord.java | 25 -- .../nifi/processors/standard/TestSplitRecord.java | 15 - 2 files changed, 37 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java index 2a5679d..d9e7bb5 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java @@ -26,6 +26,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.InputRequirement; @@ -40,6 +41,7 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.flowfile.attributes.FragmentAttributes; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; @@ -66,11 +68,20 @@ import org.apache.nifi.serialization.record.RecordSet; @Tags({"split", "generic", "schema", "json", "csv", "avro", "log", "logs", "freeform", "text"}) @WritesAttributes({ @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer for the FlowFiles routed to the 'splits' Relationship."), -@WritesAttribute(attribute = "record.count", description = "The number of records in the FlowFile. This is added to FlowFiles that are routed to the 'splits' Relationship.") +@WritesAttribute(attribute = "record.count", description = "The number of records in the FlowFile. This is added to FlowFiles that are routed to the 'splits' Relationship."), +@WritesAttribute(attribute = "fragment.identifier", description = "All split FlowFiles produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute"), +@WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the split FlowFiles that were created from a single parent FlowFile"), +@WritesAttribute(attribute = "fragment.count", description = "The number of split FlowFiles generated from the parent FlowFile"), +@WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile") }) @CapabilityDescription("Splits up an input FlowFile that is in a record-oriented data format into multiple smaller FlowFiles") public class SplitRecord extends AbstractProcessor { +public static final String FRAGMENT_ID = FragmentAttributes.FRAGMENT_ID.key(); +public static final String FRAGMENT_INDEX = FragmentAttributes.FRAGMENT_INDEX.key(); +public static final String FRAGMENT_COUNT = FragmentAttributes.FRAGMENT_COUNT.key(); +public static final String SEGMENT_ORIGINAL_FILENAME = FragmentAttributes.SEGMENT_ORIGINAL_FILENAME.key(); + static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder() .name("Record Reader") .description("Specifies the Controller Service to use for reading incoming data") @@ -125,7 +136,7 @@ public class SplitRecord extends AbstractProcessor { @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { -FlowFile original = session.get(); +final FlowFile original = session.get(); if (original == null) { return; } @@ -137,6 +148,7 @@ public class SplitRecord extends AbstractProcessor { final List split
nifi git commit: NIFI-5889 - changed the wording about Destination URL in S2S RTs
Repository: nifi Updated Branches: refs/heads/master 13a7f1686 -> 85cc5aa9e NIFI-5889 - changed the wording about Destination URL in S2S RTs This closes #3215. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/85cc5aa9 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/85cc5aa9 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/85cc5aa9 Branch: refs/heads/master Commit: 85cc5aa9eb99fce8268a476fa8a645184478be1d Parents: 13a7f16 Author: Pierre Villard Authored: Tue Dec 11 17:36:45 2018 +0100 Committer: Koji Kawamura Committed: Wed Dec 12 11:15:48 2018 +0900 -- .../apache/nifi/reporting/AbstractSiteToSiteReportingTask.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/85cc5aa9/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java -- 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/AbstractSiteToSiteReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java index 21bb397..8fdf483 100644 --- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java +++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java @@ -95,8 +95,9 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT static final PropertyDescriptor DESTINATION_URL = new PropertyDescriptor.Builder() .name("Destination URL") .displayName("Destination URL") -.description("The URL of the destination NiFi instance to send data to, " + -"should be a comma-separated list of address in the format of http(s)://host:port/nifi.") +.description("The URL of the destination NiFi instance or, if clustered, a comma-separated list of address in the format " ++ "of http(s)://host:port/nifi. This destination URL will only be used to initiate the Site-to-Site connection. The " ++ "data sent by this reporting task will be load-balanced on all the nodes of the destination (if clustered).") .required(true) .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .addValidator(new NiFiUrlValidator())
nifi git commit: NIFI-5890 Added a unit test that proves that 1.9 fixes a bug in the date conversion behavior in 1.8.
Repository: nifi Updated Branches: refs/heads/master 1a937b651 -> 13a7f1686 NIFI-5890 Added a unit test that proves that 1.9 fixes a bug in the date conversion behavior in 1.8. This closes #3214. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/13a7f168 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/13a7f168 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/13a7f168 Branch: refs/heads/master Commit: 13a7f168604f20da3c837ce3bf91a3472a9708f1 Parents: 1a937b6 Author: Mike Thomsen Authored: Tue Dec 11 11:18:52 2018 -0500 Committer: Koji Kawamura Committed: Wed Dec 12 11:08:18 2018 +0900 -- .../nifi/serialization/record/TestDataTypeUtils.java | 15 +++ 1 file changed, 15 insertions(+) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/13a7f168/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java -- 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 5f15fc2..45b65b4 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 @@ -54,6 +54,21 @@ public class TestDataTypeUtils { assertEquals("Times didn't match", ts.getTime(), sDate.getTime()); } +/* + * This was a bug in NiFi 1.8 where converting from a Timestamp to a Date with the record path API + * would throw an exception. + */ +@Test +public void testTimestampToDate() { +java.util.Date date = new java.util.Date(); +Timestamp ts = DataTypeUtils.toTimestamp(date, null, null); +assertNotNull(ts); + +java.sql.Date output = DataTypeUtils.toDate(ts, null, null); +assertNotNull(output); +assertEquals("Timestamps didn't match", output.getTime(), ts.getTime()); +} + @Test public void testConvertRecordMapToJavaMap() { assertNull(DataTypeUtils.convertRecordMapToJavaMap(null, null));
nifi git commit: NIFI-5875 Improve docs around the PriorityAttributePrioritizer
Repository: nifi Updated Branches: refs/heads/master 1ac5b9314 -> 1a937b651 NIFI-5875 Improve docs around the PriorityAttributePrioritizer Clear up the documentation around the PriorityAttributePrioritizer to make it clearer that if the attributes are both number the largest wins but if they are not numbers they sort the other way around. I had to go and look at the code to work out what was going on after reading the existing documentation. Simplify PriorityAttributePrioritzer docs Simplify the main section of documentation for PriorityAttributePrioritzer and then add a notes section with more details about the ordering. Fix example. Put back comment setting the priority attribute Further improvements to the Priority ordering This closes #3205. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/1a937b65 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/1a937b65 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/1a937b65 Branch: refs/heads/master Commit: 1a937b6573d77255984f9856bbc63f2c22f0 Parents: 1ac5b93 Author: Wil Selwood Authored: Thu Dec 6 10:17:19 2018 + Committer: Koji Kawamura Committed: Tue Dec 11 10:43:25 2018 +0900 -- nifi-docs/src/main/asciidoc/user-guide.adoc | 6 +- 1 file changed, 5 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/1a937b65/nifi-docs/src/main/asciidoc/user-guide.adoc -- diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc index 873b949..ba62327 100644 --- a/nifi-docs/src/main/asciidoc/user-guide.adoc +++ b/nifi-docs/src/main/asciidoc/user-guide.adoc @@ -1091,7 +1091,11 @@ The following prioritizers are available: - *FirstInFirstOutPrioritizer*: Given two FlowFiles, the one that reached the connection first will be processed first. - *NewestFlowFileFirstPrioritizer*: Given two FlowFiles, the one that is newest in the dataflow will be processed first. - *OldestFlowFileFirstPrioritizer*: Given two FlowFiles, the one that is oldest in the dataflow will be processed first. 'This is the default scheme that is used if no prioritizers are selected'. -- *PriorityAttributePrioritizer*: Given two FlowFiles that both have a "priority" attribute, the one that has the highest priority value will be processed first. Note that an UpdateAttribute processor should be used to add the "priority" attribute to the FlowFiles before they reach a connection that has this prioritizer set. Values for the "priority" attribute may be alphanumeric, where "a" is a higher priority than "z", and "1" is a higher priority than "9", for example. +- *PriorityAttributePrioritizer*: Given two FlowFiles, an attribute called âpriorityâ will be extracted. The one that has the lowest priority value will be processed first. +** Note that an UpdateAttribute processor should be used to add the "priority" attribute to the FlowFiles before they reach a connection that has this prioritizer set. +** If only one has that attribute it will go first. +** Values for the "priority" attribute can be alphanumeric, where "a" will come before "z" and "1" before "9" +** If "priority" attribute cannot be parsed as a long, unicode string ordering will be used. For example: "99" and "100" will be ordered so the flowfile with "99" comes first, but "A-99" and "A-100" will sort so the flowfile with "A-100" comes first. NOTE: With a <> configured, the connection has a queue per node in addition to the local queue. The prioritizer will sort the data in each queue independently.
nifi git commit: NIFI-5884 Bumping hbase-client version from 1.1.2 to 1.1.13
Repository: nifi Updated Branches: refs/heads/master e76c54234 -> 60064a9f6 NIFI-5884 Bumping hbase-client version from 1.1.2 to 1.1.13 This closes #3211. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/60064a9f Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/60064a9f Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/60064a9f Branch: refs/heads/master Commit: 60064a9f6874f56bc946dfd71c2568528040de0c Parents: e76c542 Author: Bryan Bende Authored: Fri Dec 7 10:50:43 2018 -0500 Committer: Koji Kawamura Committed: Mon Dec 10 11:00:08 2018 +0900 -- .../nifi-hbase_1_1_2-client-service/pom.xml | 2 +- .../java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/60064a9f/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/pom.xml -- diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/pom.xml index 4d7744d..6b1d5ba 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/pom.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/pom.xml @@ -25,7 +25,7 @@ nifi-hbase_1_1_2-client-service jar -1.1.2 +1.1.13 http://git-wip-us.apache.org/repos/asf/nifi/blob/60064a9f/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java -- diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java index 050c8d0..9425846 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service/src/main/java/org/apache/nifi/hbase/HBase_1_1_2_ClientService.java @@ -79,8 +79,9 @@ import java.util.concurrent.atomic.AtomicReference; @RequiresInstanceClassLoading @Tags({ "hbase", "client"}) -@CapabilityDescription("Implementation of HBaseClientService for HBase 1.1.2. This service can be configured by providing " + -"a comma-separated list of configuration files, or by specifying values for the other properties. If configuration files " + +@CapabilityDescription("Implementation of HBaseClientService using the HBase 1.1.x client. Although this service was originally built with the 1.1.2 " + +"client and has 1_1_2 in it's name, the client library has since been upgraded to 1.1.13 to leverage bug fixes. This service can be configured " + +"by providing a comma-separated list of configuration files, or by specifying values for the other properties. If configuration files " + "are provided, they will be loaded first, and the values of the additional properties will override the values from " + "the configuration files. In addition, any user defined properties on the processor will also be passed to the HBase " + "configuration.")
nifi git commit: NIFI-5881: Enable to export the template for non-ascii name
Repository: nifi Updated Branches: refs/heads/master ecbed6a78 -> e76c54234 NIFI-5881: Enable to export the template for non-ascii name This closes #3210. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/e76c5423 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/e76c5423 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/e76c5423 Branch: refs/heads/master Commit: e76c54234dc8997e22aa0227f969b5808eccacbf Parents: ecbed6a Author: Kemix Koo Authored: Fri Dec 7 16:31:36 2018 +0800 Committer: Koji Kawamura Committed: Mon Dec 10 09:51:46 2018 +0900 -- .../org/apache/nifi/web/api/TemplateResource.java | 16 ++-- 1 file changed, 14 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/e76c5423/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java index 4a3de4f..cd3605c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/TemplateResource.java @@ -43,9 +43,13 @@ import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.Context; +import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; import java.nio.charset.StandardCharsets; +import java.nio.charset.Charset; import java.util.Set; /** @@ -142,14 +146,22 @@ public class TemplateResource extends ApplicationResource { attachmentName = attachmentName.replaceAll("\\s", "_"); } +final Charset utf8 = StandardCharsets.UTF_8; +try { +attachmentName = URLEncoder.encode(attachmentName, utf8.name()); +} catch (UnsupportedEncodingException e) { +// +} + // generate the response /* * Here instead of relying on default JAXB marshalling we are simply * serializing template to String (formatted, indented etc) and sending * it as part of the response. */ -String serializedTemplate = new String(TemplateSerializer.serialize(template), StandardCharsets.UTF_8); -return generateOkResponse(serializedTemplate).header("Content-Disposition", String.format("attachment; filename=\"%s.xml\"", attachmentName)).build(); +String serializedTemplate = new String(TemplateSerializer.serialize(template), utf8); +String filename = attachmentName + ".xml"; +return generateOkResponse(serializedTemplate).encoding(utf8.name()).header(HttpHeaders.CONTENT_DISPOSITION, "attachment; filename* = " + utf8.name() + "''" + filename).build(); } /**
nifi git commit: NIFI-5862 MockRecordParser Has Bad Logic for failAfterN
Repository: nifi Updated Branches: refs/heads/master 986a2a484 -> 6c1c9017e NIFI-5862 MockRecordParser Has Bad Logic for failAfterN This closes #3195. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6c1c9017 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6c1c9017 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6c1c9017 Branch: refs/heads/master Commit: 6c1c9017e98b8467296bc90746dab73a60cb9c1b Parents: 986a2a4 Author: Peter Wicks Authored: Mon Dec 3 10:44:01 2018 -0700 Committer: Koji Kawamura Committed: Wed Dec 5 10:09:31 2018 +0900 -- .../org/apache/nifi/serialization/record/MockRecordParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/6c1c9017/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java -- diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java index 2f7c634..be4046c 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java @@ -79,7 +79,7 @@ public class MockRecordParser extends AbstractControllerService implements Recor @Override public Record nextRecord(final boolean coerceTypes, final boolean dropUnknown) throws IOException, MalformedRecordException { -if (failAfterN >= recordCount) { +if (failAfterN >= 0 && recordCount >= failAfterN) { throw new MalformedRecordException("Intentional Unit Test Exception because " + recordCount + " records have been read"); } recordCount++;
nifi git commit: NIFI-5838 - Improve the schema validation method in Kite processors
Repository: nifi Updated Branches: refs/heads/master cc9f89b00 -> 986a2a484 NIFI-5838 - Improve the schema validation method in Kite processors review Add empty check This closes #3182. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/986a2a48 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/986a2a48 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/986a2a48 Branch: refs/heads/master Commit: 986a2a484285a342e20494107abe52ff98ad2880 Parents: cc9f89b Author: Pierre Villard Authored: Thu Nov 22 18:50:11 2018 +0100 Committer: Koji Kawamura Committed: Wed Dec 5 09:53:52 2018 +0900 -- .../processors/kite/AbstractKiteProcessor.java | 25 .../processors/kite/TestCSVToAvroProcessor.java | 18 ++ 2 files changed, 33 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/986a2a48/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java -- diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java index 65dcd5f..345c1c2 100644 --- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java +++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java @@ -39,6 +39,7 @@ import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processors.hadoop.HadoopValidators; +import org.apache.nifi.util.StringUtils; import org.kitesdk.data.DatasetNotFoundException; import org.kitesdk.data.Datasets; import org.kitesdk.data.SchemaNotFoundException; @@ -101,29 +102,30 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { return parseSchema(uriOrLiteral); } +if(uri.getScheme() == null) { +throw new SchemaNotFoundException("If the schema is not a JSON string, a scheme must be specified in the URI " ++ "(ex: dataset:, view:, resource:, file:, hdfs:, etc)."); +} + try { if ("dataset".equals(uri.getScheme()) || "view".equals(uri.getScheme())) { return Datasets.load(uri).getDataset().getDescriptor().getSchema(); } else if ("resource".equals(uri.getScheme())) { -try (InputStream in = Resources.getResource(uri.getSchemeSpecificPart()) -.openStream()) { +try (InputStream in = Resources.getResource(uri.getSchemeSpecificPart()).openStream()) { return parseSchema(uri, in); } } else { // try to open the file Path schemaPath = new Path(uri); -FileSystem fs = schemaPath.getFileSystem(conf); -try (InputStream in = fs.open(schemaPath)) { +try (FileSystem fs = schemaPath.getFileSystem(conf); InputStream in = fs.open(schemaPath)) { return parseSchema(uri, in); } } } catch (DatasetNotFoundException e) { -throw new SchemaNotFoundException( -"Cannot read schema of missing dataset: " + uri, e); +throw new SchemaNotFoundException("Cannot read schema of missing dataset: " + uri, e); } catch (IOException e) { -throw new SchemaNotFoundException( -"Failed while reading " + uri + ": " + e.getMessage(), e); +throw new SchemaNotFoundException("Failed while reading " + uri + ": " + e.getMessage(), e); } } @@ -131,8 +133,7 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { try { return new Schema.Parser().parse(literal); } catch (RuntimeException e) { -throw new SchemaNotFoundException( -"Failed to parse schema: " + literal, e); +throw new SchemaNotFoundException("Failed to parse schema: " + literal, e); } } @@ -150,6 +151,10 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { Configuration conf = getConfiguration(context.getProperty(CONF_XML_FILES).evaluateAttributeExpressions().getValue()); String error = null; +if(StringUtils.isBlank(uri)) { +
nifi git commit: NIFI-5845: Add support for OTHER and SQLXML JDBC types to SQL/Hive processors
Repository: nifi Updated Branches: refs/heads/master 98cdb50cc -> 68a49cfad NIFI-5845: Add support for OTHER and SQLXML JDBC types to SQL/Hive processors NIFI-5845: Incorporated review comments This closes #3184. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/68a49cfa Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/68a49cfa Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/68a49cfa Branch: refs/heads/master Commit: 68a49cfad04df2f467f9529c0fc1b6daf1781192 Parents: 98cdb50 Author: Matthew Burgess Authored: Tue Nov 27 18:35:29 2018 -0500 Committer: Koji Kawamura Committed: Thu Nov 29 09:50:21 2018 +0900 -- .../org/apache/nifi/util/hive/HiveJdbcCommon.java | 13 + .../org/apache/nifi/util/hive/HiveJdbcCommon.java | 14 ++ .../nifi/processors/standard/util/JdbcCommon.java | 7 +++ 3 files changed, 34 insertions(+) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/68a49cfa/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java -- diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java index 6e28f94..9f07b18 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java @@ -40,6 +40,7 @@ import java.nio.charset.StandardCharsets; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.sql.SQLException; +import java.sql.SQLXML; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -64,9 +65,11 @@ import static java.sql.Types.LONGVARCHAR; import static java.sql.Types.NCHAR; import static java.sql.Types.NUMERIC; import static java.sql.Types.NVARCHAR; +import static java.sql.Types.OTHER; import static java.sql.Types.REAL; import static java.sql.Types.ROWID; import static java.sql.Types.SMALLINT; +import static java.sql.Types.SQLXML; import static java.sql.Types.STRUCT; import static java.sql.Types.TIME; import static java.sql.Types.TIMESTAMP; @@ -165,6 +168,8 @@ public class HiveJdbcCommon { } else if (value instanceof Boolean) { rec.put(i - 1, value); +} else if (value instanceof java.sql.SQLXML) { +rec.put(i - 1, ((java.sql.SQLXML) value).getString()); } else { // The different types that we support are numbers (int, long, double, float), // as well as boolean values and Strings. Since Avro doesn't provide @@ -241,6 +246,8 @@ public class HiveJdbcCommon { case ARRAY: case STRUCT: case JAVA_OBJECT: +case OTHER: +case SQLXML: builder.name(columnName).type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault(); break; @@ -402,6 +409,12 @@ public class HiveJdbcCommon { rowValues.add(""); } break; +case SQLXML: +if (value != null) { + rowValues.add(StringEscapeUtils.escapeCsv(((java.sql.SQLXML) value).getString())); +} else { +rowValues.add(""); +} default: if (value != null) { rowValues.add(value.toString()); http://git-wip-us.apache.org/repos/asf/nifi/blob/68a49cfa/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java -- diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java index 6e28f94..09eecce 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java @@ -40,6 +40,7 @@ import java.nio.charset.StandardCharsets; import java.sql.ResultSet;
nifi git commit: NIFI-5834: Restore default PutHiveQL error handling behavior
Repository: nifi Updated Branches: refs/heads/master e603c486f -> 455e3c1bc NIFI-5834: Restore default PutHiveQL error handling behavior NIFI-5834: Incorporated review comments This closes #3179. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/455e3c1b Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/455e3c1b Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/455e3c1b Branch: refs/heads/master Commit: 455e3c1bc8d4e12ea75d4e6ac2e4c58cbb535e5d Parents: e603c48 Author: Matthew Burgess Authored: Tue Nov 20 17:58:59 2018 -0500 Committer: Koji Kawamura Committed: Tue Nov 27 18:10:26 2018 +0900 -- .../apache/nifi/processors/hive/PutHiveQL.java | 7 +- .../nifi/processors/hive/TestPutHiveQL.java | 80 +++- .../apache/nifi/processors/hive/PutHive3QL.java | 19 - .../nifi/processors/hive/TestPutHive3QL.java| 79 ++- 4 files changed, 179 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/455e3c1b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java -- diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java index e053a9a..bb5d526 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java @@ -149,10 +149,11 @@ public class PutHiveQL extends AbstractHiveQLProcessor { } else if (e instanceof SQLException) { // Use the SQLException's vendor code for guidance -- see Hive's ErrorMsg class for details on error codes int errorCode = ((SQLException) e).getErrorCode(); +getLogger().debug("Error occurred during Hive operation, Hive returned error code {}", new Object[]{errorCode}); if (errorCode >= 1 && errorCode < 2) { return ErrorTypes.InvalidInput; } else if (errorCode >= 2 && errorCode < 3) { -return ErrorTypes.TemporalFailure; +return ErrorTypes.InvalidInput; } else if (errorCode >= 3 && errorCode < 4) { return ErrorTypes.TemporalInputFailure; } else if (errorCode >= 4 && errorCode < 5) { @@ -160,7 +161,9 @@ public class PutHiveQL extends AbstractHiveQLProcessor { // a ProcessException, we'll route to failure via an InvalidInput error type. return ErrorTypes.InvalidInput; } else { -return ErrorTypes.UnknownFailure; +// Default unknown errors to TemporalFailure (as they were implemented originally), so they can be routed to failure +// or rolled back depending on the user's setting of Rollback On Failure. +return ErrorTypes.TemporalFailure; } } else { return ErrorTypes.UnknownFailure; http://git-wip-us.apache.org/repos/asf/nifi/blob/455e3c1b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveQL.java -- diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveQL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveQL.java index af737ae..dd16ca1 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveQL.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHiveQL.java @@ -719,6 +719,78 @@ public class TestPutHiveQL { runner.assertAllFlowFilesTransferred(PutHiveQL.REL_RETRY, 0); } +@Test +public void testUnknownFailure() throws InitializationException, ProcessException { +final TestRunner runner = TestRunners.newTestRunner(PutHiveQL.class); +final SQLExceptionService service = new SQLExceptionService(null); +service.setErrorCode(2); +runner.addControllerService("dbcp", service); +runner.enableControllerService(service); + +runner.setProperty(PutHiveQL.HIVE_DBCP_SERVICE, "dbcp"); + +
nifi git commit: NIFI-5828: Documents behavior of ExecuteSQL attrs when Max Rows Per Flow File is enabled
Repository: nifi Updated Branches: refs/heads/master 023f0c41c -> 54402a1ec NIFI-5828: Documents behavior of ExecuteSQL attrs when Max Rows Per Flow File is enabled This closes #3177. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/54402a1e Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/54402a1e Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/54402a1e Branch: refs/heads/master Commit: 54402a1ecda4b11ca10472fada9adce484b031c1 Parents: 023f0c4 Author: Colin Dean Authored: Sun Nov 18 23:14:55 2018 -0500 Committer: Koji Kawamura Committed: Mon Nov 19 14:18:31 2018 +0900 -- .../org/apache/nifi/processors/standard/ExecuteSQL.java | 12 1 file changed, 8 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/54402a1e/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java index 9c61793..99e0d2a 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java @@ -77,10 +77,14 @@ import static org.apache.nifi.processors.standard.util.JdbcCommon.USE_AVRO_LOGIC + "'-MM-dd HH:mm:ss.SSS' for Timestamp is used.") }) @WritesAttributes({ -@WritesAttribute(attribute = "executesql.row.count", description = "Contains the number of rows returned in the select query"), -@WritesAttribute(attribute = "executesql.query.duration", description = "Combined duration of the query execution time and fetch time in milliseconds"), -@WritesAttribute(attribute = "executesql.query.executiontime", description = "Duration of the query execution time in milliseconds"), -@WritesAttribute(attribute = "executesql.query.fetchtime", description = "Duration of the result set fetch time in milliseconds"), +@WritesAttribute(attribute = "executesql.row.count", description = "Contains the number of rows returned by the query. " ++ "If 'Max Rows Per Flow File' is set, then this number will reflect the number of rows in the Flow File instead of the entire result set."), +@WritesAttribute(attribute = "executesql.query.duration", description = "Combined duration of the query execution time and fetch time in milliseconds. " ++ "If 'Max Rows Per Flow File' is set, then this number will reflect only the fetch time for the rows in the Flow File instead of the entire result set."), +@WritesAttribute(attribute = "executesql.query.executiontime", description = "Duration of the query execution time in milliseconds. " ++ "This number will reflect the query execution time regardless of the 'Max Rows Per Flow File' setting."), +@WritesAttribute(attribute = "executesql.query.fetchtime", description = "Duration of the result set fetch time in milliseconds. " ++ "If 'Max Rows Per Flow File' is set, then this number will reflect only the fetch time for the rows in the Flow File instead of the entire result set."), @WritesAttribute(attribute = "executesql.resultset.index", description = "Assuming multiple result sets are returned, " + "the zero based index of this result set."), @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Rows Per Flow File' is set then all FlowFiles from the same query result set "
nifi git commit: NIFI-5823: Fixes typo in min idle connections property name
Repository: nifi Updated Branches: refs/heads/master 0207d0813 -> 102a5288e NIFI-5823: Fixes typo in min idle connections property name So many people missed it :-( This closes #3172. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/102a5288 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/102a5288 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/102a5288 Branch: refs/heads/master Commit: 102a5288efb2a22cd54815dd7331dfc5826aee91 Parents: 0207d08 Author: Colin Dean Authored: Thu Nov 15 13:46:41 2018 -0500 Committer: Koji Kawamura Committed: Fri Nov 16 11:16:55 2018 +0900 -- .../src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/102a5288/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java -- diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java index 5228496..d6f83f2 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java @@ -194,7 +194,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder() .displayName("Minimum Idle Connections") -.name("dbcp-mim-idle-conns") +.name("dbcp-min-idle-conns") .description("The minimum number of connections that can remain idle in the pool, without extra ones being " + "created, or zero to create none.") .defaultValue(DEFAULT_MIN_IDLE) @@ -438,4 +438,4 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC BasicDataSource getDataSource() { return dataSource; } -} \ No newline at end of file +}
nifi git commit: NIFI-5652: Fixed LogMessage when logging level is disabled
Repository: nifi Updated Branches: refs/heads/master 4112af013 -> 13011ac6d NIFI-5652: Fixed LogMessage when logging level is disabled This closes #3170. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/13011ac6 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/13011ac6 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/13011ac6 Branch: refs/heads/master Commit: 13011ac6d61961ecd3f3524b1e0dfda382ed4dd6 Parents: 4112af0 Author: Matthew Burgess Authored: Wed Nov 14 10:50:37 2018 -0500 Committer: Koji Kawamura Committed: Thu Nov 15 15:14:57 2018 +0900 -- .../apache/nifi/processors/standard/LogMessage.java | 15 ++- 1 file changed, 2 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/13011ac6/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java index 5ec07eb..152117f 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java @@ -156,12 +156,9 @@ public class LogMessage extends AbstractProcessor { break; } -if (!isLogLevelEnabled) { -transferChunk(session); -return; +if (isLogLevelEnabled) { +processFlowFile(logger, logLevel, flowFile, context); } - -processFlowFile(logger, logLevel, flowFile, context); session.transfer(flowFile, REL_SUCCESS); } @@ -202,12 +199,4 @@ public class LogMessage extends AbstractProcessor { logger.debug(messageToWrite); } } - -private void transferChunk(final ProcessSession session) { -final List flowFiles = session.get(CHUNK_SIZE); -if (!flowFiles.isEmpty()) { -session.transfer(flowFiles, REL_SUCCESS); -} -} - }
nifi git commit: NIFI-5815 - PutORC processor 'Restricted' still requires access to restricted components regardless of restriction
Repository: nifi Updated Branches: refs/heads/master d3b167481 -> 9e7610ac7 NIFI-5815 - PutORC processor 'Restricted' still requires access to restricted components regardless of restriction This closes #3169. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/9e7610ac Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/9e7610ac Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/9e7610ac Branch: refs/heads/master Commit: 9e7610ac70c389ecfb3a4d389e4e28a41249af2b Parents: d3b1674 Author: Pierre Villard Authored: Mon Nov 12 20:22:03 2018 +0100 Committer: Koji Kawamura Committed: Wed Nov 14 13:50:00 2018 +0900 -- .../src/main/java/org/apache/nifi/processors/orc/PutORC.java | 8 +++- 1 file changed, 7 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/9e7610ac/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java -- diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java index 9af566a..8dfbc96 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/orc/PutORC.java @@ -25,12 +25,14 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.behavior.Restricted; +import org.apache.nifi.annotation.behavior.Restriction; import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.RequiredPermission; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.DataUnit; @@ -63,7 +65,11 @@ import java.util.List; + "the path is the directory that contains this ORC file on HDFS. For example, this processor can send flow files downstream to ReplaceText to set the content " + "to this DDL (plus the LOCATION clause as described), then to PutHiveQL processor to create the table if it doesn't exist.") }) -@Restricted("Provides operator the ability to write to any file that NiFi has access to in HDFS or the local filesystem.") +@Restricted(restrictions = { +@Restriction( +requiredPermission = RequiredPermission.WRITE_FILESYSTEM, +explanation = "Provides operator the ability to write to any file that NiFi has access to in HDFS or the local filesystem.") +}) public class PutORC extends AbstractPutHDFSRecord { public static final String HIVE_DDL_ATTRIBUTE = "hive.ddl";
nifi git commit: NIFI-5809: If QueryRecord has a single-column projection and that results in a null value, do not confuse that with a null value being returned from the Record Reader
Repository: nifi Updated Branches: refs/heads/master 878a0b8b7 -> 08189596d NIFI-5809: If QueryRecord has a single-column projection and that results in a null value, do not confuse that with a null value being returned from the Record Reader This closes #3163. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/08189596 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/08189596 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/08189596 Branch: refs/heads/master Commit: 08189596d27af7bb4518646245549b480d4bb05a Parents: 878a0b8 Author: Mark Payne Authored: Fri Nov 9 11:40:59 2018 -0500 Committer: Koji Kawamura Committed: Mon Nov 12 14:39:27 2018 +0900 -- .../nifi/queryrecord/FlowFileEnumerator.java| 38 +++-- .../processors/standard/TestQueryRecord.java| 57 2 files changed, 64 insertions(+), 31 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/08189596/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java index 963b85e..5f92311 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java @@ -17,8 +17,6 @@ package org.apache.nifi.queryrecord; -import java.io.InputStream; - import org.apache.calcite.linq4j.Enumerator; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; @@ -28,6 +26,8 @@ import org.apache.nifi.serialization.RecordReader; import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.record.Record; +import java.io.InputStream; + public class FlowFileEnumerator implements Enumerator { private final ProcessSession session; private final FlowFile flowFile; @@ -57,26 +57,24 @@ public class FlowFileEnumerator implements Enumerator { @Override public boolean moveNext() { currentRow = null; -while (currentRow == null) { -try { -currentRow = filterColumns(recordParser.nextRecord()); -break; -} catch (final Exception e) { -throw new ProcessException("Failed to read next record in stream for " + flowFile + " due to " + e.getMessage(), e); -} -} - -if (currentRow == null) { -// If we are out of data, close the InputStream. We do this because -// Calcite does not necessarily call our close() method. -close(); -try { -onFinish(); -} catch (final Exception e) { -logger.error("Failed to perform tasks when enumerator was finished", e); +try { +final Record record = recordParser.nextRecord(); +if (record == null) { +// If we are out of data, close the InputStream. We do this because +// Calcite does not necessarily call our close() method. +close(); +try { +onFinish(); +} catch (final Exception e) { +logger.error("Failed to perform tasks when enumerator was finished", e); +} + +return false; } -return false; +currentRow = filterColumns(record); +} catch (final Exception e) { +throw new ProcessException("Failed to read next record in stream for " + flowFile + " due to " + e.getMessage(), e); } recordsRead++; http://git-wip-us.apache.org/repos/asf/nifi/blob/08189596/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java index 60fefef..ce710f5 100644 ---
nifi git commit: NIFI-5724 make database connection autocommit configurable
Repository: nifi Updated Branches: refs/heads/master d5bce9197 -> 63f55d05b NIFI-5724 make database connection autocommit configurable making the database session autocommit value a configurable property adding custom validation to PutSQL processor so as to disallow 'supports transaction' and 'rollback on failure' to be true when the autocommit value has been set to true fixing some style issues to conform to standards This closes #3113. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/63f55d05 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/63f55d05 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/63f55d05 Branch: refs/heads/master Commit: 63f55d05b4f3f83b9e9f2206f4129ae7a4ade569 Parents: d5bce91 Author: Vish Uma Authored: Fri Oct 26 15:32:46 2018 -0400 Committer: Koji Kawamura Committed: Fri Nov 9 17:07:27 2018 +0900 -- .../apache/nifi/processors/standard/PutSQL.java | 53 ++-- 1 file changed, 49 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/63f55d05/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java index 9957c2e..38134c2 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java @@ -28,6 +28,8 @@ import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.dbcp.DBCPService; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; @@ -64,6 +66,7 @@ import java.sql.SQLNonTransientException; import java.sql.Statement; import java.util.ArrayList; import java.util.BitSet; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -74,6 +77,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; +import static java.lang.String.format; import static org.apache.nifi.processor.util.pattern.ExceptionHandler.createOnError; @SupportsBatching @@ -134,6 +138,14 @@ public class PutSQL extends AbstractSessionFactoryProcessor { .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .build(); +static final PropertyDescriptor AUTO_COMMIT = new PropertyDescriptor.Builder() +.name("database-session-autocommit") +.displayName("Database Session AutoCommit") +.description("The autocommit mode to set on the database connection being used.") +.allowableValues("true", "false") +.defaultValue("false") +.build(); + static final PropertyDescriptor SUPPORT_TRANSACTIONS = new PropertyDescriptor.Builder() .name("Support Fragmented Transactions") .description("If true, when a FlowFile is consumed by this Processor, the Processor will first check the fragment.identifier and fragment.count attributes of that FlowFile. " @@ -189,6 +201,7 @@ public class PutSQL extends AbstractSessionFactoryProcessor { properties.add(CONNECTION_POOL); properties.add(SQL_STATEMENT); properties.add(SUPPORT_TRANSACTIONS); +properties.add(AUTO_COMMIT); properties.add(TRANSACTION_TIMEOUT); properties.add(BATCH_SIZE); properties.add(OBTAIN_GENERATED_KEYS); @@ -197,6 +210,34 @@ public class PutSQL extends AbstractSessionFactoryProcessor { } @Override +protected final Collection customValidate(ValidationContext context) { +final Collection results = new ArrayList<>(); +final String support_transactions = context.getProperty(SUPPORT_TRANSACTIONS).getValue(); +final String rollback_on_failure = context.getProperty(RollbackOnFailure.ROLLBACK_ON_FAILURE).getValue(); +final String auto_commit = context.getProperty(AUTO_COMMIT).getValue(); + +if(auto_commit.equalsIgnoreCase("true")) { +
nifi git commit: NIFI-5794 Allow empty string demarcator in ConsumeKafka processors
Repository: nifi Updated Branches: refs/heads/master 49ba3643c -> d5bce9197 NIFI-5794 Allow empty string demarcator in ConsumeKafka processors This closes #3159. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d5bce919 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d5bce919 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d5bce919 Branch: refs/heads/master Commit: d5bce91970b287009dc09d0ce197a537dc0be207 Parents: 49ba364 Author: Pierre Villard Authored: Wed Nov 7 10:55:10 2018 +0100 Committer: Koji Kawamura Committed: Fri Nov 9 09:41:45 2018 +0900 -- .../apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java| 3 ++- .../apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java| 3 ++- .../org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java | 3 ++- .../org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java | 3 ++- .../org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_0.java | 3 ++- 5 files changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/d5bce919/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java -- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java index f652c28..0d80e76 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java @@ -43,6 +43,7 @@ import org.apache.nifi.components.AllowableValue; 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.logging.ComponentLog; import org.apache.nifi.processor.AbstractProcessor; @@ -132,7 +133,7 @@ public class ConsumeKafka_0_10 extends AbstractProcessor { .name("message-demarcator") .displayName("Message Demarcator") .required(false) -.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.addValidator(Validator.VALID) .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .description("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains " + "all Kafka messages in a single batch for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use " http://git-wip-us.apache.org/repos/asf/nifi/blob/d5bce919/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java -- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java index 5927e7e..1a627dc 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java @@ -44,6 +44,7 @@ import org.apache.nifi.components.AllowableValue; 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.logging.ComponentLog; import org.apache.nifi.processor.AbstractProcessor; @@ -133,7 +134,7 @@ public class ConsumeKafka_0_11 extends AbstractProcessor { .name("message-demarcator") .displayName("Message Demarcator") .required(false) -.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.addValidator(Validator.VALID) .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
nifi git commit: NIFI-5752: Load balancing fails with wildcard certs
Repository: nifi Updated Branches: refs/heads/master da1f9eaf6 -> 13232c741 NIFI-5752: Load balancing fails with wildcard certs NIFI-5752: Remove an unnecessary String.format NIFI-5752: Remove an unnecessary block This closes #3110. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/13232c74 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/13232c74 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/13232c74 Branch: refs/heads/master Commit: 13232c74136e8452b3cbd708e535af7a1fc0d1cb Parents: da1f9ea Author: Kotaro Terada Authored: Thu Oct 25 19:46:06 2018 +0900 Committer: Koji Kawamura Committed: Wed Nov 7 17:16:20 2018 +0900 -- .../server/ClusterLoadBalanceAuthorizer.java| 55 +--- .../clustered/server/LoadBalanceAuthorizer.java | 11 ++-- .../server/StandardLoadBalanceProtocol.java | 39 ++ .../queue/clustered/LoadBalancedQueueIT.java| 4 +- .../server/TestStandardLoadBalanceProtocol.java | 2 +- 5 files changed, 60 insertions(+), 51 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/13232c74/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ClusterLoadBalanceAuthorizer.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ClusterLoadBalanceAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ClusterLoadBalanceAuthorizer.java index f0d51c8..fbd849c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ClusterLoadBalanceAuthorizer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ClusterLoadBalanceAuthorizer.java @@ -17,14 +17,23 @@ package org.apache.nifi.controller.queue.clustered.server; +import org.apache.http.conn.ssl.DefaultHostnameVerifier; import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.events.EventReporter; import org.apache.nifi.reporting.Severity; +import org.apache.nifi.security.util.CertificateUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collection; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLSession; +import javax.net.ssl.SSLSocket; +import java.io.IOException; +import java.security.cert.Certificate; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; import java.util.Set; import java.util.stream.Collectors; @@ -33,19 +42,27 @@ public class ClusterLoadBalanceAuthorizer implements LoadBalanceAuthorizer { private final ClusterCoordinator clusterCoordinator; private final EventReporter eventReporter; +private final HostnameVerifier hostnameVerifier; public ClusterLoadBalanceAuthorizer(final ClusterCoordinator clusterCoordinator, final EventReporter eventReporter) { this.clusterCoordinator = clusterCoordinator; this.eventReporter = eventReporter; +this.hostnameVerifier = new DefaultHostnameVerifier(); } @Override -public String authorize(final Collection clientIdentities) throws NotAuthorizedException { -if (clientIdentities == null) { -logger.debug("Client Identities is null, so assuming that Load Balancing communications are not secure. Authorizing client to participate in Load Balancing"); -return null; +public String authorize(SSLSocket sslSocket) throws NotAuthorizedException, IOException { +final SSLSession sslSession = sslSocket.getSession(); + +final Set clientIdentities; +try { +clientIdentities = getCertificateIdentities(sslSession); +} catch (final CertificateException e) { +throw new IOException("Failed to extract Client Certificate", e); } +logger.debug("Will perform authorization against Client Identities '{}'", clientIdentities); + final Set nodeIds = clusterCoordinator.getNodeIdentifiers().stream() .map(NodeIdentifier::getApiAddress) .collect(Collectors.toSet()); @@ -57,11 +74,35 @@ public class ClusterLoadBalanceAuthorizer implements LoadBalanceAuthorizer { } } -final String message =
nifi git commit: NIFI-5777: Update the log-level property of LogMessage to support EL
Repository: nifi Updated Branches: refs/heads/master aabb5d783 -> 0c9120a73 NIFI-5777: Update the log-level property of LogMessage to support EL This closes #3122. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/0c9120a7 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/0c9120a7 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/0c9120a7 Branch: refs/heads/master Commit: 0c9120a73fe99bc4ced50997ee9d65250fb7cf3a Parents: aabb5d7 Author: Kotaro Terada Authored: Tue Nov 6 14:26:50 2018 +0900 Committer: Koji Kawamura Committed: Tue Nov 6 16:20:40 2018 +0900 -- .../java/org/apache/nifi/processors/standard/LogMessage.java| 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/0c9120a7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java index d8e33ca..5ec07eb 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LogMessage.java @@ -37,6 +37,7 @@ import org.apache.nifi.processor.util.StandardValidators; import org.eclipse.jetty.util.StringUtil; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -54,8 +55,8 @@ public class LogMessage extends AbstractProcessor { .name("log-level") .displayName("Log Level") .required(true) -.description("The Log Level to use when logging the message") -.allowableValues(MessageLogLevel.values()) +.description("The Log Level to use when logging the message: " + Arrays.toString(MessageLogLevel.values())) +.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .defaultValue(MessageLogLevel.info.toString()) .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .build();
nifi git commit: NIFI-5787 Update ReadMe doc to start Nifi on windows
Repository: nifi Updated Branches: refs/heads/master d0688d078 -> aabb5d783 NIFI-5787 Update ReadMe doc to start Nifi on windows Update 1.8 assembly ReadMe doc to start Nifi on windows. Change it to run-nifi.bat This closes #3127. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/aabb5d78 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/aabb5d78 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/aabb5d78 Branch: refs/heads/master Commit: aabb5d7832cbedb54d385db9d75f565ae3043dc7 Parents: d0688d0 Author: Brandon Authored: Mon Nov 5 11:16:37 2018 +0800 Committer: Koji Kawamura Committed: Tue Nov 6 15:49:03 2018 +0900 -- nifi-assembly/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/aabb5d78/nifi-assembly/README.md -- diff --git a/nifi-assembly/README.md b/nifi-assembly/README.md index 6e86801..c39f51f 100644 --- a/nifi-assembly/README.md +++ b/nifi-assembly/README.md @@ -50,7 +50,7 @@ Apache NiFi was made for dataflow. It supports highly configurable directed grap To start NiFi: - [linux/osx] execute bin/nifi.sh start -- [windows] execute bin/start-nifi.bat +- [windows] execute bin/run-nifi.bat - Direct your browser to http://localhost:8080/nifi/ ## Getting Help
nifi git commit: NIFI-5677 Added note to clarify why modifying/creating variables not local changes in versioned flows
Repository: nifi Updated Branches: refs/heads/master 37a0e1b30 -> d0688d078 NIFI-5677 Added note to clarify why modifying/creating variables not local changes in versioned flows Update user-guide.adoc This closes #3125. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d0688d07 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d0688d07 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d0688d07 Branch: refs/heads/master Commit: d0688d0785dd7dee0e761975f993bdaf8e374139 Parents: 37a0e1b Author: Andrew Lim Authored: Fri Nov 2 13:12:05 2018 -0400 Committer: Koji Kawamura Committed: Tue Nov 6 15:17:32 2018 +0900 -- nifi-docs/src/main/asciidoc/user-guide.adoc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/d0688d07/nifi-docs/src/main/asciidoc/user-guide.adoc -- diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc index c69b771..873b949 100644 --- a/nifi-docs/src/main/asciidoc/user-guide.adoc +++ b/nifi-docs/src/main/asciidoc/user-guide.adoc @@ -1928,7 +1928,9 @@ The following actions are not considered local changes: * modifying sensitive property values * modifying remote process group URLs * updating a processor that was referencing a non-existent controller service to reference an externally available controller service -* modifying variables +* creating, modifying or deleting variables + +NOTE: Creating a variable does not trigger a local change because creating a variable on its own has not changed anything about what the flow processes. A component will have to be created or modified that uses the new variable, which will trigger a local change. Modifying a variable does not trigger a local change because variable values are intended to be different in each environment. When a versioned flow is imported, it is assumed there is a one-time operation required to set those variables specific for the given environment. Deleting a variable does not trigger a local change because the component that references that variable will need need to be modified, which will trigger a local change. WARNING: Variables do not support sensitive values and will be included when versioning a Process Group. See <> for more information.
nifi git commit: NIFI-5761 ReplaceText processor can stop processing data if it evaluates invalid expressions
Repository: nifi Updated Branches: refs/heads/master fdb1fd1a6 -> 1f2cf4bc6 NIFI-5761 ReplaceText processor can stop processing data if it evaluates invalid expressions NIFI-5761 Code review. Remove startsWidth to check an exception. Added the dependency as provided. NIFI-5761 Code review. Remove provided. NIFI-5761 Code review. Improve logging. This closes #3112. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/1f2cf4bc Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/1f2cf4bc Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/1f2cf4bc Branch: refs/heads/master Commit: 1f2cf4bc6128e2a09dcc378bdf66b46302d44754 Parents: fdb1fd1 Author: Gardella Juan Pablo Authored: Fri Oct 26 16:41:26 2018 -0300 Committer: Koji Kawamura Committed: Wed Oct 31 12:01:16 2018 +0900 -- .../nifi-standard-processors/pom.xml| 4 .../nifi/processors/standard/ReplaceText.java | 10 +--- .../processors/standard/TestReplaceText.java| 25 nifi-nar-bundles/nifi-standard-bundle/pom.xml | 5 4 files changed, 41 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/1f2cf4bc/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml -- 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 ed86dfd..620c570 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 @@ -26,6 +26,10 @@ org.apache.nifi +nifi-expression-language + + +org.apache.nifi nifi-processor-utils 1.9.0-SNAPSHOT http://git-wip-us.apache.org/repos/asf/nifi/blob/1f2cf4bc/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java -- 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 2a9a6fe..3108a6c 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,8 @@ 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.attribute.expression.language.exception.AttributeExpressionLanguageException; +import org.apache.nifi.attribute.expression.language.exception.IllegalAttributeException; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; @@ -297,17 +299,19 @@ public class ReplaceText extends AbstractProcessor { } catch (StackOverflowError e) { // Some regular expressions can produce many matches on large input data size using recursive code // do not log the StackOverflowError stack trace -logger.info("Transferred {} to 'failure' due to {}", new Object[] {flowFile, e.toString()}); +logger.info("Transferred {} to 'failure' due to {}", new Object[] { flowFile, e.toString() }); +session.transfer(flowFile, REL_FAILURE); +return; +} catch (IllegalAttributeException | AttributeExpressionLanguageException e) { +logger.warn("Transferred {} to 'failure' due to {}", new Object[] { flowFile, e.toString() }, e); 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); } - // If we find a back reference that is not valid, then we will treat it as a literal string. For example, if we have 3 capturing // groups and the Replacement Value has the value is "I owe $8 to him", then we want to treat the $8 as a literal "$8", rather // than
nifi git commit: NIFI-5765 Fixing WriteJsonResult to use chosenDataType when writing an Array value
Repository: nifi Updated Branches: refs/heads/master f6d8eada6 -> db966cf34 NIFI-5765 Fixing WriteJsonResult to use chosenDataType when writing an Array value Fixing the same bug in PutElasticsearchHttpRecord.java This closes #3114. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/db966cf3 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/db966cf3 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/db966cf3 Branch: refs/heads/master Commit: db966cf34879ea0d750ae7ab72344b228f889f59 Parents: f6d8ead Author: Joe Percivall Authored: Mon Oct 29 16:04:05 2018 -0400 Committer: Koji Kawamura Committed: Tue Oct 30 11:51:17 2018 +0900 -- .../PutElasticsearchHttpRecord.java | 2 +- .../org/apache/nifi/json/WriteJsonResult.java | 22 .../apache/nifi/json/TestWriteJsonResult.java | 59 +++- 3 files changed, 56 insertions(+), 27 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/db966cf3/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java -- diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java index 2448716..ac36604 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java @@ -569,7 +569,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess default: if (coercedValue instanceof Object[]) { final Object[] values = (Object[]) coercedValue; -final ArrayDataType arrayDataType = (ArrayDataType) dataType; +final ArrayDataType arrayDataType = (ArrayDataType) chosenDataType; final DataType elementType = arrayDataType.getElementType(); writeArray(values, fieldName, generator, elementType); } else { http://git-wip-us.apache.org/repos/asf/nifi/blob/db966cf3/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java -- 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/WriteJsonResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java index d113f8d..5708f5e 100755 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java @@ -17,17 +17,8 @@ package org.apache.nifi.json; -import java.io.IOException; -import java.io.OutputStream; -import java.math.BigInteger; -import java.text.DateFormat; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.function.Supplier; - -import org.apache.nifi.record.NullSuppression; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.record.NullSuppression; import org.apache.nifi.schema.access.SchemaAccessWriter; import org.apache.nifi.serialization.AbstractRecordSetWriter; import org.apache.nifi.serialization.RecordSetWriter; @@ -49,6 +40,15 @@ import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.JsonGenerator; import org.codehaus.jackson.util.MinimalPrettyPrinter; +import java.io.IOException; +import java.io.OutputStream; +import java.math.BigInteger; +import java.text.DateFormat; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; + public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSetWriter, RawRecordWriter { private final ComponentLog logger; private final SchemaAccessWriter
nifi git commit: NIFI-5747 fix NPE when redirecting from HTTPS to HTTP for InvokeHTTP
Repository: nifi Updated Branches: refs/heads/master 234ddb0fe -> fdbcf3428 NIFI-5747 fix NPE when redirecting from HTTPS to HTTP for InvokeHTTP Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/fdbcf342 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/fdbcf342 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/fdbcf342 Branch: refs/heads/master Commit: fdbcf34281bf6ee71f4f5ba6b9445254bc874752 Parents: 234ddb0 Author: juliansniff Authored: Wed Oct 24 19:47:40 2018 -0400 Committer: Koji Kawamura Committed: Fri Oct 26 17:08:17 2018 +0900 -- .../org/apache/nifi/processors/standard/InvokeHTTP.java | 10 +++--- 1 file changed, 7 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/fdbcf342/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java -- 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 7d04698..8926ba2 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 @@ -21,6 +21,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.security.Principal; import okhttp3.Cache; import okhttp3.Credentials; import okhttp3.MediaType; @@ -132,7 +133,6 @@ import static org.apache.commons.lang3.StringUtils.trimToEmpty; description = "Send request header with a key matching the Dynamic Property Key and a value created by evaluating " + "the Attribute Expression Language set in the value of the Dynamic Property.") public final class InvokeHTTP extends AbstractProcessor { - // flowfile attribute keys returned after reading the response public final static String STATUS_CODE = "invokehttp.status.code"; public final static String STATUS_MESSAGE = "invokehttp.status.message"; @@ -1182,8 +1182,12 @@ public final class InvokeHTTP extends AbstractProcessor { map.put(key, value); }); -if ("HTTPS".equals(url.getProtocol().toUpperCase())) { -map.put(REMOTE_DN, responseHttp.handshake().peerPrincipal().getName()); +if (responseHttp.request().isHttps()) { +Principal principal = responseHttp.handshake().peerPrincipal(); + +if (principal != null) { +map.put(REMOTE_DN, principal.getName()); +} } return map;
nifi git commit: NIFI-5745: When determining if backpressure should be applied across nodes for load balancing, only consider if the local partition has reached the threshold limits instead of conside
Repository: nifi Updated Branches: refs/heads/master c7ff2fc5d -> 234ddb0fe NIFI-5745: When determining if backpressure should be applied across nodes for load balancing, only consider if the local partition has reached the threshold limits instead of considering the size of the entire queue This closes #3108. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/234ddb0f Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/234ddb0f Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/234ddb0f Branch: refs/heads/master Commit: 234ddb0fe1a36ad947c340114058d82c777d791f Parents: c7ff2fc Author: Mark Payne Authored: Wed Oct 24 13:06:57 2018 -0400 Committer: Koji Kawamura Committed: Fri Oct 26 16:04:46 2018 +0900 -- .../nifi/controller/queue/LoadBalancedFlowFileQueue.java | 6 ++ .../apache/nifi/controller/queue/AbstractFlowFileQueue.java| 5 - .../queue/clustered/SocketLoadBalancedFlowFileQueue.java | 5 + .../queue/clustered/server/StandardLoadBalanceProtocol.java| 4 +++- 4 files changed, 18 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/234ddb0f/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LoadBalancedFlowFileQueue.java -- diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LoadBalancedFlowFileQueue.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LoadBalancedFlowFileQueue.java index f0eff27..b9f6951 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LoadBalancedFlowFileQueue.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LoadBalancedFlowFileQueue.java @@ -66,4 +66,10 @@ public interface LoadBalancedFlowFileQueue extends FlowFileQueue { */ boolean isPropagateBackpressureAcrossNodes(); +/** + * Determines whether or not the local partition's size >= backpressure threshold + * + * @return true if the number of FlowFiles or total size of FlowFiles in the local partition alone meets or exceeds the backpressure threshold, false otherwise. + */ +boolean isLocalPartitionFull(); } http://git-wip-us.apache.org/repos/asf/nifi/blob/234ddb0f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java index 5bf75a4..436b85d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java @@ -141,6 +141,10 @@ public abstract class AbstractFlowFileQueue implements FlowFileQueue { @Override public boolean isFull() { +return isFull(size()); +} + +protected boolean isFull(final QueueSize queueSize) { final MaxQueueSize maxSize = getMaxQueueSize(); // Check if max size is set @@ -148,7 +152,6 @@ public abstract class AbstractFlowFileQueue implements FlowFileQueue { return false; } -final QueueSize queueSize = size(); if (maxSize.getMaxCount() > 0 && queueSize.getObjectCount() >= maxSize.getMaxCount()) { return true; } http://git-wip-us.apache.org/repos/asf/nifi/blob/234ddb0f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java index 7b3a211..84731f7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java +++
nifi git commit: NIFI-5746: Use Node Identifier's node address instead of getting from socket for RECEIVE prov events; make SEND prov events match syntax of RECEIVE prov events
Repository: nifi Updated Branches: refs/heads/master 2201f7746 -> c7ff2fc5d NIFI-5746: Use Node Identifier's node address instead of getting from socket for RECEIVE prov events; make SEND prov events match syntax of RECEIVE prov events NIFI-5746: Addressed issue found in review process This closes #3109. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c7ff2fc5 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c7ff2fc5 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c7ff2fc5 Branch: refs/heads/master Commit: c7ff2fc5dba7c8aaeae07f4819c320e2a96555f0 Parents: 2201f77 Author: Mark Payne Authored: Wed Oct 24 13:45:55 2018 -0400 Committer: Koji Kawamura Committed: Fri Oct 26 10:24:12 2018 +0900 -- .../client/async/TransactionCompleteCallback.java | 3 ++- .../async/nio/NioAsyncLoadBalanceClient.java | 2 +- .../clustered/partition/RemoteQueuePartition.java | 15 --- .../server/ClusterLoadBalanceAuthorizer.java | 6 +++--- .../clustered/server/LoadBalanceAuthorizer.java | 10 +- .../server/StandardLoadBalanceProtocol.java | 17 +++-- .../queue/clustered/LoadBalancedQueueIT.java | 4 ++-- .../client/async/nio/TestLoadBalanceSession.java | 4 ++-- .../server/TestStandardLoadBalanceProtocol.java | 18 +- 9 files changed, 43 insertions(+), 36 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/c7ff2fc5/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/TransactionCompleteCallback.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/TransactionCompleteCallback.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/TransactionCompleteCallback.java index 0c8f8b6..6e327cd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/TransactionCompleteCallback.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/TransactionCompleteCallback.java @@ -17,10 +17,11 @@ package org.apache.nifi.controller.queue.clustered.client.async; +import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.controller.repository.FlowFileRecord; import java.util.List; public interface TransactionCompleteCallback { -void onTransactionComplete(List flowFilesSent); +void onTransactionComplete(List flowFilesSent, NodeIdentifier nodeIdentifier); } http://git-wip-us.apache.org/repos/asf/nifi/blob/c7ff2fc5/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClient.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClient.java index 753c1f4..e55dfcd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClient.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClient.java @@ -256,7 +256,7 @@ public class NioAsyncLoadBalanceClient implements AsyncLoadBalanceClient { } while (success); if (loadBalanceSession.isComplete()) { - loadBalanceSession.getPartition().getSuccessCallback().onTransactionComplete(loadBalanceSession.getFlowFilesSent()); + loadBalanceSession.getPartition().getSuccessCallback().onTransactionComplete(loadBalanceSession.getFlowFilesSent(), nodeIdentifier); } return anySuccess; http://git-wip-us.apache.org/repos/asf/nifi/blob/c7ff2fc5/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/RemoteQueuePartition.java
nifi git commit: NIFI-5446: Specify the file encoding option to UTF-8 in pom.xml
Repository: nifi Updated Branches: refs/heads/master f3f7cdbab -> ec8afd18d NIFI-5446: Specify the file encoding option to UTF-8 in pom.xml This closes #2910. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/ec8afd18 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/ec8afd18 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/ec8afd18 Branch: refs/heads/master Commit: ec8afd18d6ae39dc1b09698fabfd1d3e7eaa70cf Parents: f3f7cdb Author: Takanobu Asanuma Authored: Mon Jul 23 13:53:51 2018 +0900 Committer: Koji Kawamura Committed: Thu Oct 18 15:09:23 2018 +0900 -- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/ec8afd18/pom.xml -- diff --git a/pom.xml b/pom.xml index b86cada..37c3c01 100644 --- a/pom.xml +++ b/pom.xml @@ -348,7 +348,8 @@ true -Xmx1G -Djava.net.preferIPv4Stack=true -${maven.surefire.arguments} +${maven.surefire.arguments} +-Dfile.encoding=UTF-8
nifi git commit: NIFI-5719 Ensuring FetchFile routes to failure if the move completion strategy can't be completed
Repository: nifi Updated Branches: refs/heads/master 5ec85299e -> f3f7cdbab NIFI-5719 Ensuring FetchFile routes to failure if the move completion strategy can't be completed This closes #3088. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/f3f7cdba Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/f3f7cdba Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/f3f7cdba Branch: refs/heads/master Commit: f3f7cdbab9248ba13c00d0e27a398e820e9ec7ed Parents: 5ec8529 Author: Bryan Bende Authored: Wed Oct 17 11:28:29 2018 -0400 Committer: Koji Kawamura Committed: Thu Oct 18 14:37:55 2018 +0900 -- .../nifi/processors/standard/FetchFile.java | 12 .../nifi/processors/standard/TestFetchFile.java | 73 2 files changed, 85 insertions(+) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/f3f7cdba/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java index 3bf3f52..b929f07 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java @@ -236,6 +236,18 @@ public class FetchFile extends AbstractProcessor { return; } +if (!targetDir.exists()) { +try { +Files.createDirectories(targetDir.toPath()); +} catch (Exception e) { +getLogger().error("Could not fetch file {} from file system for {} because Completion Strategy is configured to move the original file to {}, " ++ "but that directory does not exist and could not be created due to: {}", +new Object[] {file, flowFile, targetDir, e.getMessage()}, e); +session.transfer(flowFile, REL_FAILURE); +return; +} +} + final String conflictStrategy = context.getProperty(CONFLICT_STRATEGY).getValue(); if (CONFLICT_FAIL.getValue().equalsIgnoreCase(conflictStrategy)) { http://git-wip-us.apache.org/repos/asf/nifi/blob/f3f7cdba/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFile.java -- 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 cfced0a..dcafefe 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 @@ -42,6 +42,8 @@ public class TestFetchFile { return; } +targetDir.setReadable(true); + for (final File file : targetDir.listFiles()) { Files.delete(file.toPath()); } @@ -110,6 +112,9 @@ public class TestFetchFile { runner.assertValid(); final File destDir = new File("target/move-target"); +destDir.mkdirs(); +assertTrue(destDir.exists()); + final File destFile = new File(destDir, sourceFile.getName()); runner.enqueue(new byte[0]); @@ -135,6 +140,11 @@ public class TestFetchFile { runner.assertValid(); final File destDir = new File("target/move-target"); +if (destDir.exists()) { +destDir.delete(); +} +assertFalse(destDir.exists()); + final File destFile = new File(destDir, sourceFile.getName()); runner.enqueue(new byte[0]); @@ -147,6 +157,69 @@ public class TestFetchFile { } @Test +public void testMoveOnCompleteWithTargetExistsButNotWritable() throws IOException { +final File sourceFile = new File("target/1.txt"); +final byte[] content = "Hello, World!".getBytes(); +Files.write(sourceFile.toPath(), content,
nifi git commit: NIFI-5696 Update references to default value for nifi.cluster.node.load.load.balance.port
Repository: nifi Updated Branches: refs/heads/master 0229a5c10 -> 7cb39d636 NIFI-5696 Update references to default value for nifi.cluster.node.load.load.balance.port This closes #3071. Signed-off-by: Koji Kawamura Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/7cb39d63 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/7cb39d63 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/7cb39d63 Branch: refs/heads/master Commit: 7cb39d636c33289a1f93417c1750ef6173df348e Parents: 0229a5c Author: Jeff Storck Authored: Fri Oct 12 16:57:15 2018 -0400 Committer: Koji Kawamura Committed: Mon Oct 15 10:43:31 2018 +0900 -- .../java/org/apache/nifi/cluster/protocol/NodeIdentifier.java| 4 +++- .../nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/7cb39d63/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java index 56f9a24..74c5538 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java @@ -17,6 +17,7 @@ package org.apache.nifi.cluster.protocol; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.util.NiFiProperties; import javax.xml.bind.annotation.XmlAccessType; import javax.xml.bind.annotation.XmlAccessorType; @@ -107,7 +108,8 @@ public class NodeIdentifier { public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort, final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure) { -this(id, apiAddress, apiPort, socketAddress, socketPort, socketAddress, 6342, siteToSiteAddress, siteToSitePort, siteToSiteHttpApiPort, siteToSiteSecure, null); +this(id, apiAddress, apiPort, socketAddress, socketPort, socketAddress, NiFiProperties.DEFAULT_LOAD_BALANCE_PORT, siteToSiteAddress, siteToSitePort, siteToSiteHttpApiPort, siteToSiteSecure, +null); } public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort, final String loadBalanceAddress, final int loadBalancePort, http://git-wip-us.apache.org/repos/asf/nifi/blob/7cb39d63/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml index c186e54..fc18be8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml @@ -185,7 +185,7 @@ -7430 +6342 4 8 30 sec