[1/2] incubator-nifi git commit: NIFI-685: This is merged. This closes #67
Repository: incubator-nifi Updated Branches: refs/heads/develop 8ff69ca2d - 483b3dddf NIFI-685: This is merged. This closes #67 Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/4dc126af Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/4dc126af Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/4dc126af Branch: refs/heads/develop Commit: 4dc126aff8751e66018ac03260f3061b97719487 Parents: 8bd2051 Author: Mark Payne marka...@hotmail.com Authored: Tue Jul 14 08:40:04 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jul 14 08:40:04 2015 -0400 -- --
[2/2] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/483b3ddd Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/483b3ddd Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/483b3ddd Branch: refs/heads/develop Commit: 483b3dddfb7fc0cb569f8553513286d1d2cf1cf2 Parents: 4dc126a 8ff69ca Author: Mark Payne marka...@hotmail.com Authored: Tue Jul 14 08:40:22 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jul 14 08:40:22 2015 -0400 -- .../processors/kite/AvroRecordConverter.java| 320 + .../nifi/processors/kite/ConvertAvroSchema.java | 339 +++ .../org.apache.nifi.processor.Processor | 1 + .../additionalDetails.html | 142 .../kite/TestAvroRecordConverter.java | 201 +++ .../processors/kite/TestConvertAvroSchema.java | 216 6 files changed, 1219 insertions(+) --
incubator-nifi git commit: NIFI-762: Allow user to set keystore and truststore properties instead of setting sslcontext
Repository: incubator-nifi Updated Branches: refs/heads/develop 7f6f404ba - 8bd20510e NIFI-762: Allow user to set keystore and truststore properties instead of setting sslcontext Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/8bd20510 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/8bd20510 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/8bd20510 Branch: refs/heads/develop Commit: 8bd20510ee348e497b6039e083b32e37a4d3a20e Parents: 7f6f404 Author: Mark Payne marka...@hotmail.com Authored: Mon Jul 13 14:17:42 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jul 13 14:17:42 2015 -0400 -- .../org/apache/nifi/events/EventReporter.java | 4 +- .../apache/nifi/remote/client/KeystoreType.java | 24 ++ .../nifi/remote/client/SiteToSiteClient.java| 257 ++- .../remote/client/SiteToSiteClientConfig.java | 32 ++- .../apache/nifi/controller/FlowController.java | 2 + .../nifi/remote/StandardRemoteProcessGroup.java | 2 + .../nifi/remote/StandardRootGroupPort.java | 2 + .../TestPersistentProvenanceRepository.java | 2 + 8 files changed, 311 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8bd20510/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java b/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java index 76702f1..d645d60 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java @@ -16,12 +16,14 @@ */ package org.apache.nifi.events; +import java.io.Serializable; + import org.apache.nifi.reporting.Severity; /** * Implementations MUST be thread-safe */ -public interface EventReporter { +public interface EventReporter extends Serializable { void reportEvent(Severity severity, String category, String message); } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8bd20510/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java -- diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java new file mode 100644 index 000..63c3d63 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the License); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.remote.client; + +import java.io.Serializable; + +public enum KeystoreType implements Serializable { +PKCS12, +JKS; +} http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8bd20510/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java -- diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java index 5c4ce55..78237b9 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java @@ -18,11 +18,17 @@ package org.apache.nifi.remote.client; import java.io.Closeable; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; +import java.io.InputStream; import java.io.Serializable; +import java.security.KeyStore; +import java.security.SecureRandom;
[2/2] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/3125036d Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/3125036d Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/3125036d Branch: refs/heads/develop Commit: 3125036d3f463c97a10466d244952f10f0054b64 Parents: c0f1104 9fa03b4 Author: Mark Payne marka...@hotmail.com Authored: Tue Jul 7 21:08:34 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jul 7 21:08:34 2015 -0400 -- nifi/nifi-external/nifi-storm-spout/pom.xml | 2 +- .../java/org/apache/nifi/storm/NiFiSpout.java | 38 ++-- .../ConfigurableComponentInitializer.java | 9 - .../apache/nifi/documentation/DocGenerator.java | 2 ++ .../init/ControllerServiceInitializer.java | 11 -- .../init/ProcessorInitializer.java | 11 -- .../init/ReportingTaskingInitializer.java | 11 -- .../html/HtmlDocumentationWriterTest.java | 10 +++--- .../html/ProcessorDocumentationWriterTest.java | 7 ++-- 9 files changed, 82 insertions(+), 19 deletions(-) --
[1/2] incubator-nifi git commit: NIFI-733: Make use of Client Name, Zookeeper Timeout, Kafka Timeout properties and add new Group ID property
Repository: incubator-nifi Updated Branches: refs/heads/develop b17184946 - 322ac6fba NIFI-733: Make use of Client Name, Zookeeper Timeout, Kafka Timeout properties and add new Group ID property Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/786bc1d6 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/786bc1d6 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/786bc1d6 Branch: refs/heads/develop Commit: 786bc1d61260e2d8558747ca206d360aebdd1994 Parents: 063afe2 Author: Mark Payne marka...@hotmail.com Authored: Fri Jul 3 09:04:52 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 09:04:52 2015 -0400 -- .../apache/nifi/processors/kafka/GetKafka.java | 28 +++- 1 file changed, 22 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/786bc1d6/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java -- diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java index 1b63a46..26590df 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java @@ -129,6 +129,7 @@ public class GetKafka extends AbstractProcessor { .expressionLanguageSupported(false) .defaultValue(\\n) .build(); + public static final PropertyDescriptor CLIENT_NAME = new PropertyDescriptor.Builder() .name(Client Name) .description(Client Name to use when communicating with Kafka) @@ -136,6 +137,13 @@ public class GetKafka extends AbstractProcessor { .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(false) .build(); +public static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder() +.name(Group ID) +.description(A Group ID is used to identify consumers that are within the same consumer group) +.required(true) +.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.expressionLanguageSupported(false) +.build(); public static final Relationship REL_SUCCESS = new Relationship.Builder() .name(success) @@ -152,9 +160,13 @@ public class GetKafka extends AbstractProcessor { @Override protected ListPropertyDescriptor getSupportedPropertyDescriptors() { final PropertyDescriptor clientNameWithDefault = new PropertyDescriptor.Builder() -.fromPropertyDescriptor(CLIENT_NAME) -.defaultValue(NiFi- + getIdentifier()) -.build(); +.fromPropertyDescriptor(CLIENT_NAME) +.defaultValue(NiFi- + getIdentifier()) +.build(); +final PropertyDescriptor groupIdWithDefault = new PropertyDescriptor.Builder() +.fromPropertyDescriptor(GROUP_ID) +.defaultValue(getIdentifier()) +.build(); final ListPropertyDescriptor props = new ArrayList(); props.add(ZOOKEEPER_CONNECTION_STRING); @@ -163,6 +175,7 @@ public class GetKafka extends AbstractProcessor { props.add(BATCH_SIZE); props.add(MESSAGE_DEMARCATOR); props.add(clientNameWithDefault); +props.add(groupIdWithDefault); props.add(KAFKA_TIMEOUT); props.add(ZOOKEEPER_TIMEOUT); return props; @@ -184,10 +197,13 @@ public class GetKafka extends AbstractProcessor { final Properties props = new Properties(); props.setProperty(zookeeper.connect, context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue()); -props.setProperty(group.id, getIdentifier()); +props.setProperty(group.id, context.getProperty(GROUP_ID).getValue()); +props.setProperty(client.id, context.getProperty(CLIENT_NAME).getValue()); props.setProperty(auto.commit.interval.ms, String.valueOf(context.getProperty(ZOOKEEPER_COMMIT_DELAY).asTimePeriod(TimeUnit.MILLISECONDS))); props.setProperty(auto.commit.enable, true); // just be explicit props.setProperty(auto.offset.reset, smallest); +props.setProperty(zk.connectiontimeout.ms, context.getProperty(ZOOKEEPER_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).toString()); +props.setProperty(socket.timeout.ms,
[2/2] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/322ac6fb Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/322ac6fb Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/322ac6fb Branch: refs/heads/develop Commit: 322ac6fba6bec0e691b29cea74af7a93572677a5 Parents: 786bc1d b171849 Author: Mark Payne marka...@hotmail.com Authored: Fri Jul 3 09:38:19 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 09:38:19 2015 -0400 -- .../nifi/processors/standard/InvokeHTTP.java| 46 +++- .../processors/standard/TestInvokeHTTP.java | 109 +++ 2 files changed, 154 insertions(+), 1 deletion(-) --
[1/2] incubator-nifi git commit: NIFI-717: Set working directory to $NIFI_HOME before running starting nifi in linux
Repository: incubator-nifi Updated Branches: refs/heads/develop 63a900852 - f3b55d46d NIFI-717: Set working directory to $NIFI_HOME before running starting nifi in linux Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a2b9d70a Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a2b9d70a Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a2b9d70a Branch: refs/heads/develop Commit: a2b9d70ac43f6f3162abb7706f9021ce3e3993f0 Parents: 8da7327 Author: Mark Payne marka...@hotmail.com Authored: Fri Jul 3 13:33:06 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 13:33:06 2015 -0400 -- .../nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a2b9d70a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh index 01a3f81..86e97d8 100755 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh @@ -183,9 +183,9 @@ run() { # run 'start' in the background because the process will continue to run, monitoring NiFi. # all other commands will terminate quickly so want to just wait for them if [ $1 = start ]; then -(${sudo_cmd_prefix} $JAVA -cp $NIFI_HOME/conf/:$NIFI_HOME/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file=$BOOTSTRAP_CONF org.apache.nifi.bootstrap.RunNiFi $@ ) +(cd $NIFI_HOME ${sudo_cmd_prefix} $JAVA -cp $NIFI_HOME/conf/:$NIFI_HOME/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file=$BOOTSTRAP_CONF org.apache.nifi.bootstrap.RunNiFi $@ ) else -${sudo_cmd_prefix} $JAVA -cp $NIFI_HOME/conf/:$NIFI_HOME/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file=$BOOTSTRAP_CONF org.apache.nifi.bootstrap.RunNiFi $@ +(cd $NIFI_HOME ${sudo_cmd_prefix} $JAVA -cp $NIFI_HOME/conf/:$NIFI_HOME/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file=$BOOTSTRAP_CONF org.apache.nifi.bootstrap.RunNiFi $@) fi # Wait just a bit (3 secs) to wait for the logging to finish and then echo a new-line.
incubator-nifi git commit: NIFI-743 Moving OnShutdown to OnStopped for GetSolr and GetHttp
Repository: incubator-nifi Updated Branches: refs/heads/develop 8da732718 - 63a900852 NIFI-743 Moving OnShutdown to OnStopped for GetSolr and GetHttp Adding OnRemoved methods to GetSolr and GetHttp Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/63a90085 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/63a90085 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/63a90085 Branch: refs/heads/develop Commit: 63a900852769bbd7bedc2b460a291a9de0bd9a15 Parents: 8da7327 Author: Bryan Bende bbe...@apache.org Authored: Fri Jul 3 10:52:35 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 13:37:44 2015 -0400 -- .../apache/nifi/processors/solr/GetSolr.java| 15 -- .../nifi/processors/solr/TestGetSolr.java | 29 +--- .../nifi/processors/standard/GetHTTP.java | 15 -- .../nifi/processors/standard/TestGetHTTP.java | 10 +-- 4 files changed, 56 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/63a90085/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java -- diff --git a/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java b/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java index 6344e2c..ff264a1 100644 --- a/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java +++ b/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/GetSolr.java @@ -21,7 +21,8 @@ package org.apache.nifi.processors.solr; import org.apache.commons.io.IOUtils; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnShutdown; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; @@ -157,11 +158,19 @@ public class GetSolr extends SolrProcessor { lastEndDatedRef.set(UNINITIALIZED_LAST_END_DATE_VALUE); } -@OnShutdown -public void onShutdown() { +@OnStopped +public void onStopped() { writeLastEndDate(); } +@OnRemoved +public void onRemoved() { +final File lastEndDateCache = new File(FILE_PREFIX + getIdentifier()); +if (lastEndDateCache.exists()) { +lastEndDateCache.delete(); +} +} + @Override public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { final ProcessorLog logger = getLogger(); http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/63a90085/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java -- diff --git a/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java b/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java index b0f5e68..d8f6423 100644 --- a/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java +++ b/nifi/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java @@ -104,14 +104,15 @@ public class TestGetSolr { File confDir = new File(conf); assertTrue(confDir.exists()); File[] files = confDir.listFiles(); -assertTrue(files.length 0); -for (File file : files) { -assertTrue(Failed to delete + file.getName(), file.delete()); +if (files.length 0) { +for (File file : files) { +assertTrue(Failed to delete + file.getName(), file.delete()); +} } assertTrue(confDir.delete()); try { -solrClient.shutdown(); +solrClient.close(); } catch (Exception e) { } } @@ -184,6 +185,26 @@ public class TestGetSolr { runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 0); } +@Test +public void testOnRemovedRemovesState() throws IOException, SolrServerException { +
[4/4] incubator-nifi git commit: NIFI-685: Fixed checkstyle violation of using import with *
NIFI-685: Fixed checkstyle violation of using import with * Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/2a4e5e12 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/2a4e5e12 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/2a4e5e12 Branch: refs/heads/develop Commit: 2a4e5e12855364c6d4b14f1ef1445e08d1f4fe08 Parents: 2ac61c5 Author: Mark Payne marka...@hotmail.com Authored: Fri Jul 3 12:55:32 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 12:55:32 2015 -0400 -- .../nifi/processors/standard/PostHTTP.java | 30 +++- 1 file changed, 17 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2a4e5e12/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java index 1dc7aba..51f28e0 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java @@ -49,7 +49,11 @@ import javax.net.ssl.SSLSession; import javax.security.cert.X509Certificate; import javax.servlet.http.HttpServletResponse; -import org.apache.http.*; +import org.apache.http.Header; +import org.apache.http.HttpException; +import org.apache.http.HttpHost; +import org.apache.http.HttpResponse; +import org.apache.http.HttpResponseInterceptor; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; @@ -78,9 +82,9 @@ import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; import org.apache.http.protocol.HttpContext; import org.apache.http.protocol.HttpCoreContext; import org.apache.http.util.EntityUtils; +import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; @@ -378,7 +382,7 @@ public class PostHTTP extends AbstractProcessor { config = new Config(conMan); final Config existingConfig = configMap.putIfAbsent(baseUrl, config); -return (existingConfig == null) ? config : existingConfig; +return existingConfig == null ? config : existingConfig; } private SSLContext createSSLContext(final SSLContextService service) @@ -402,7 +406,7 @@ public class PostHTTP extends AbstractProcessor { builder = builder.loadKeyMaterial(keystore, service.getKeyStorePassword().toCharArray()); } -SSLContext sslContext = builder.build(); +final SSLContext sslContext = builder.build(); return sslContext; } @@ -468,13 +472,13 @@ public class PostHTTP extends AbstractProcessor { clientBuilder.addInterceptorFirst(new HttpResponseInterceptor() { @Override public void process(final HttpResponse response, final HttpContext httpContext) throws HttpException, IOException { -HttpCoreContext coreContext = HttpCoreContext.adapt(httpContext); -ManagedHttpClientConnection conn = coreContext.getConnection(ManagedHttpClientConnection.class); +final HttpCoreContext coreContext = HttpCoreContext.adapt(httpContext); +final ManagedHttpClientConnection conn = coreContext.getConnection(ManagedHttpClientConnection.class); if (!conn.isOpen()) { return; } -SSLSession sslSession = conn.getSSLSession(); +final SSLSession sslSession = conn.getSSLSession(); if (sslSession != null) { final X509Certificate[] certChain = sslSession.getPeerCertificateChain(); @@ -524,7 +528,7 @@ public class PostHTTP extends AbstractProcessor { }
[3/4] incubator-nifi git commit: Adding a plain socket factory for use with regular proxies
Adding a plain socket factory for use with regular proxies Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/2ac61c59 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/2ac61c59 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/2ac61c59 Branch: refs/heads/develop Commit: 2ac61c5972d30eb3840ca6d4066f14f87e5adb00 Parents: 05772a0 Author: Andrew Hulbert andrew.hulb...@ccri.com Authored: Mon Jun 8 15:29:14 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 12:48:46 2015 -0400 -- .../java/org/apache/nifi/processors/standard/GetHTTP.java| 8 +++- .../java/org/apache/nifi/processors/standard/PostHTTP.java | 8 +++- 2 files changed, 14 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2ac61c59/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index 6e9acdd..1ecd170 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -62,6 +62,7 @@ import org.apache.http.config.Registry; import org.apache.http.config.RegistryBuilder; import org.apache.http.conn.HttpClientConnectionManager; import org.apache.http.conn.socket.ConnectionSocketFactory; +import org.apache.http.conn.socket.PlainConnectionSocketFactory; import org.apache.http.conn.ssl.SSLConnectionSocketFactory; import org.apache.http.conn.ssl.SSLContexts; import org.apache.http.conn.ssl.TrustSelfSignedStrategy; @@ -358,7 +359,12 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { final SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslContext, new String[]{TLSv1}, null, SSLConnectionSocketFactory.BROWSER_COMPATIBLE_HOSTNAME_VERIFIER); -final RegistryConnectionSocketFactory socketFactoryRegistry = RegistryBuilder.ConnectionSocketFactorycreate().register(https, sslsf).build(); +// Also include a plain socket factory for regular http connections (especially proxies) +final RegistryConnectionSocketFactory socketFactoryRegistry = +RegistryBuilder.ConnectionSocketFactorycreate() +.register(https, sslsf) +.register(http, PlainConnectionSocketFactory.getSocketFactory()) +.build(); conMan = new BasicHttpClientConnectionManager(socketFactoryRegistry); } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2ac61c59/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java index 98056cb..1dc7aba 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java @@ -64,6 +64,7 @@ import org.apache.http.config.RegistryBuilder; import org.apache.http.conn.HttpClientConnectionManager; import org.apache.http.conn.ManagedHttpClientConnection; import org.apache.http.conn.socket.ConnectionSocketFactory; +import org.apache.http.conn.socket.PlainConnectionSocketFactory; import org.apache.http.conn.ssl.SSLConnectionSocketFactory; import org.apache.http.conn.ssl.SSLContextBuilder; import org.apache.http.conn.ssl.SSLContexts; @@ -362,7 +363,12 @@ public class PostHTTP extends AbstractProcessor { final SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslContext, new String[]{TLSv1}, null, SSLConnectionSocketFactory.BROWSER_COMPATIBLE_HOSTNAME_VERIFIER); -final RegistryConnectionSocketFactory socketFactoryRegistry = RegistryBuilder.ConnectionSocketFactorycreate().register(https,
[2/4] incubator-nifi git commit: newline
newline Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/05772a07 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/05772a07 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/05772a07 Branch: refs/heads/develop Commit: 05772a07cf9a8c14143d805c9698634326d5024f Parents: 159615b Author: Andrew Hulbert andrew.hulb...@ccri.com Authored: Fri Jun 5 23:32:59 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 12:48:45 2015 -0400 -- .../main/java/org/apache/nifi/processors/standard/PostHTTP.java| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05772a07/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java index 31b5698..98056cb 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java @@ -504,7 +504,7 @@ public class PostHTTP extends AbstractProcessor { final int port = context.getProperty(PROXY_PORT).asInteger(); clientBuilder.setProxy(new HttpHost(host, port)); } - + client = clientBuilder.build(); // determine whether or not destination accepts flowfile/gzip
[1/4] incubator-nifi git commit: initial proxy server commit
Repository: incubator-nifi Updated Branches: refs/heads/develop 74b480093 - 2a4e5e128 initial proxy server commit Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/159615bb Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/159615bb Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/159615bb Branch: refs/heads/develop Commit: 159615bb8e6ca6e48a8418e9059874cdd4b4eee1 Parents: 74b4800 Author: Andrew Hulbert andrew.hulb...@ccri.com Authored: Fri Jun 5 23:10:16 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 12:48:44 2015 -0400 -- .../nifi/processors/standard/GetHTTP.java | 30 .../nifi/processors/standard/PostHTTP.java | 36 +--- 2 files changed, 61 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/159615bb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index 0001bb3..6e9acdd 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -50,6 +50,7 @@ import java.util.regex.Pattern; import javax.net.ssl.SSLContext; import org.apache.http.Header; +import org.apache.http.HttpHost; import org.apache.http.HttpResponse; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; @@ -171,6 +172,18 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { .required(false) .identifiesControllerService(SSLContextService.class) .build(); +public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder() +.name(Proxy Host) +.description(The fully qualified hostname or IP address of the proxy server) +.required(false) +.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.build(); +public static final PropertyDescriptor PROXY_PORT = new PropertyDescriptor.Builder() +.name(Proxy Port) +.description(The port of the proxy server) +.required(false) +.addValidator(StandardValidators.PORT_VALIDATOR) +.build(); public static final Relationship REL_SUCCESS = new Relationship.Builder() .name(success) @@ -222,6 +235,8 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { properties.add(USER_AGENT); properties.add(ACCEPT_CONTENT_TYPE); properties.add(FOLLOW_REDIRECTS); +properties.add(PROXY_HOST); +properties.add(PROXY_PORT); this.properties = Collections.unmodifiableList(properties); // load etag and lastModified from file @@ -276,6 +291,14 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { .build()); } +if (context.getProperty(PROXY_HOST).isSet() !context.getProperty(PROXY_PORT).isSet()) { +results.add(new ValidationResult.Builder() +.explanation(Proxy Host was set but no Proxy Port was specified) +.valid(false) +.subject(Proxy server configuration) +.build()); +} + return results; } @@ -378,6 +401,13 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { clientBuilder.setDefaultCredentialsProvider(credentialsProvider); } +// Set the proxy if specified +if (context.getProperty(PROXY_HOST).isSet() context.getProperty(PROXY_PORT).isSet()) { +final String host = context.getProperty(PROXY_HOST).getValue(); +final int port = context.getProperty(PROXY_PORT).asInteger(); +clientBuilder.setProxy(new HttpHost(host, port)); +} + // create the http client final HttpClient client = clientBuilder.build(); http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/159615bb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
incubator-nifi git commit: NIFI-720: Ensure that if Reporting Task stopped while @OnScheduled method is failing that it does not start running when configuration is fixed
Repository: incubator-nifi Updated Branches: refs/heads/develop 2a4e5e128 - 8da732718 NIFI-720: Ensure that if Reporting Task stopped while @OnScheduled method is failing that it does not start running when configuration is fixed Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/8da73271 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/8da73271 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/8da73271 Branch: refs/heads/develop Commit: 8da7327188ebdb3cbadda257429e4967be07bf77 Parents: 2a4e5e1 Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 23 16:23:01 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 13:14:16 2015 -0400 -- .../scheduling/StandardProcessScheduler.java| 74 - .../TestStandardProcessScheduler.java | 107 +++ .../src/test/resources/nifi.properties | 2 +- 3 files changed, 154 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8da73271/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index cf644ed..d976bd0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -174,31 +174,45 @@ public final class StandardProcessScheduler implements ProcessScheduler { @SuppressWarnings(deprecation) @Override public void run() { +final long lastStopTime = scheduleState.getLastStopTime(); +final ReportingTask reportingTask = taskNode.getReportingTask(); + // Continually attempt to start the Reporting Task, and if we fail sleep for a bit each time. while (true) { -final ReportingTask reportingTask = taskNode.getReportingTask(); - try { -try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.invokeMethodsWithAnnotations(OnScheduled.class, OnConfigured.class, reportingTask, taskNode.getConfigurationContext()); -} +synchronized (scheduleState) { +// if no longer scheduled to run, then we're finished. This can happen, for example, +// if the @OnScheduled method throws an Exception and the user stops the reporting task +// while we're administratively yielded. +// we also check if the schedule state's last start time is equal to what it was before. +// if not, then means that the reporting task has been stopped and started again, so we should just +// bail; another thread will be responsible for invoking the @OnScheduled methods. +if (!scheduleState.isScheduled() || scheduleState.getLastStopTime() != lastStopTime) { +return; +} + +try (final NarCloseable x = NarCloseable.withNarLoader()) { + ReflectionUtils.invokeMethodsWithAnnotations(OnScheduled.class, OnConfigured.class, reportingTask, taskNode.getConfigurationContext()); +} -break; +agent.schedule(taskNode, scheduleState); +return; +} } catch (final Exception e) { final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e; final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask); componentLog.error(Failed to invoke @OnEnabled method due to {}, cause); -LOG.error(Failed to invoke the On-Scheduled Lifecycle methods of {} due to {}; administratively yielding this
[2/2] incubator-nifi git commit: Merge branch 'NIFI-717' into develop
Merge branch 'NIFI-717' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f3b55d46 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f3b55d46 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f3b55d46 Branch: refs/heads/develop Commit: f3b55d46d6a374ee27b2decb3ee7284fd294a10d Parents: 63a9008 a2b9d70 Author: Mark Payne marka...@hotmail.com Authored: Fri Jul 3 13:50:43 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 13:50:43 2015 -0400 -- .../nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) --
incubator-nifi git commit: NIFI-745: Only call methods with @OnDisabled once, regardless of whether or not they succeed
Repository: incubator-nifi Updated Branches: refs/heads/develop f3b55d46d - 208402472 NIFI-745: Only call methods with @OnDisabled once, regardless of whether or not they succeed Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/20840247 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/20840247 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/20840247 Branch: refs/heads/develop Commit: 208402472dd99c629afc056f4464b925b8f834ab Parents: f3b55d4 Author: Mark Payne marka...@hotmail.com Authored: Fri Jul 3 14:35:59 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jul 3 15:00:31 2015 -0400 -- .../nifi/annotation/lifecycle/OnDisabled.java | 10 ++--- .../scheduling/StandardProcessScheduler.java| 39 +--- 2 files changed, 22 insertions(+), 27 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/20840247/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java index f205bc7..f8ca038 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java @@ -36,11 +36,11 @@ import org.apache.nifi.controller.ConfigurationContext; * Methods using this annotation are permitted to take zero arguments or to take * a single argument of type {@link ConfigurationContext}. If a method with this * annotation throws a Throwable, a log message and bulletin will be issued for - * the service, and the service will remain in a 'DISABLING' state. When this - * occurs, the method with this annotation will be called again after some - * period of time. This will continue until the method returns without throwing - * any Throwable. Until that time, the service will remain in a 'DISABLING' - * state and cannot be enabled again. + * the service, but the service will still be marked as Disabled. The failing + * method will not be called again until the service is enabled and disabled again. + * This is done in order to prevent a ControllerService from continually failing + * in such a way that the service could not be disabled and updated without + * restarting the instance of NiFi. * /p * * p http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/20840247/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index d976bd0..5ac4a0b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -685,33 +685,28 @@ public final class StandardProcessScheduler implements ProcessScheduler { try (final NarCloseable x = NarCloseable.withNarLoader()) { final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider, null); -while (true) { -try { - ReflectionUtils.invokeMethodsWithAnnotation(OnDisabled.class, service.getControllerServiceImplementation(), configContext); -heartbeater.heartbeat(); -service.setState(ControllerServiceState.DISABLED); -return; -} catch (final Exception e) { -final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e; -final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service); -componentLog.error(Failed to invoke @OnDisabled method due to {}, cause); - -LOG.error(Failed to invoke @OnDisabled method of {} due to {}, service.getControllerServiceImplementation(),
incubator-nifi git commit: NIFI-724: Ensure that bulletins generated for reporting tasks and controller services are shown at Controller level as well as component level
Repository: incubator-nifi Updated Branches: refs/heads/NIFI-724 59aa8ffe1 - e7c0461b1 NIFI-724: Ensure that bulletins generated for reporting tasks and controller services are shown at Controller level as well as component level Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/e7c0461b Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/e7c0461b Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/e7c0461b Branch: refs/heads/NIFI-724 Commit: e7c0461b15bff045d68e7ae8814eda2073cba209 Parents: 59aa8ff Author: Mark Payne marka...@hotmail.com Authored: Wed Jul 1 12:54:18 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jul 1 12:54:18 2015 -0400 -- .../nifi/events/VolatileBulletinRepository.java | 105 ++- 1 file changed, 78 insertions(+), 27 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e7c0461b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java index c18fffd..8aeb34d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java @@ -36,6 +36,8 @@ public class VolatileBulletinRepository implements BulletinRepository { private static final int CONTROLLER_BUFFER_SIZE = 10; private static final int COMPONENT_BUFFER_SIZE = 5; private static final String CONTROLLER_BULLETIN_STORE_KEY = CONTROLLER; +private static final String SERVICE_BULLETIN_STORE_KEY = SERVICE; +private static final String REPORTING_TASK_BULLETIN_STORE_KEY = REPORTING_TASK; private final ConcurrentMapString, ConcurrentMapString, RingBufferBulletin bulletinStoreMap = new ConcurrentHashMap(); private volatile BulletinProcessingStrategy processingStrategy = new DefaultBulletinProcessingStrategy(); @@ -170,18 +172,39 @@ public class VolatileBulletinRepository implements BulletinRepository { public ListBulletin findBulletinsForController(final int max) { final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5); -final ConcurrentMapString, RingBufferBulletin componentMap = bulletinStoreMap.get(CONTROLLER_BULLETIN_STORE_KEY); -if (componentMap == null) { -return Collections.BulletinemptyList(); -} - -final RingBufferBulletin buffer = componentMap.get(CONTROLLER_BULLETIN_STORE_KEY); -return buffer == null ? Collections.BulletinemptyList() : buffer.getSelectedElements(new FilterBulletin() { +final FilterBulletin filter = new FilterBulletin() { @Override public boolean select(final Bulletin bulletin) { return bulletin.getTimestamp().getTime() = fiveMinutesAgo; } -}, max); +}; + +final ListBulletin controllerBulletins = new ArrayList(); + +final ConcurrentMapString, RingBufferBulletin controllerBulletinMap = bulletinStoreMap.get(CONTROLLER_BULLETIN_STORE_KEY); +if (controllerBulletinMap != null) { +final RingBufferBulletin buffer = controllerBulletinMap.get(CONTROLLER_BULLETIN_STORE_KEY); +if (buffer != null) { +controllerBulletins.addAll(buffer.getSelectedElements(filter, max)); +} +} + +for (final String key : new String[] { SERVICE_BULLETIN_STORE_KEY, REPORTING_TASK_BULLETIN_STORE_KEY }) { +final ConcurrentMapString, RingBufferBulletin bulletinMap = bulletinStoreMap.get(key); +if (bulletinMap != null) { +for (final RingBufferBulletin buffer : bulletinMap.values()) { + controllerBulletins.addAll(buffer.getSelectedElements(filter, max)); +} +} +} + +// We only want the newest bulletin, so we sort based on time and take the top 'max' entries +Collections.sort(controllerBulletins); +if (controllerBulletins.size() max) { +return controllerBulletins.subList(0, max); +} + +return controllerBulletins; } /** @@ -203,7 +226,7 @@ public class
[1/2] incubator-nifi git commit: NIFI-719: Expose scheduling period to the ConfigurationContext
Repository: incubator-nifi Updated Branches: refs/heads/develop 50215707f - a09180799 NIFI-719: Expose scheduling period to the ConfigurationContext Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/ba315598 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/ba315598 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/ba315598 Branch: refs/heads/develop Commit: ba3155980efd846b45bd47eefa14f503a13908f3 Parents: e767f5c Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 23 20:47:35 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 24 08:13:06 2015 -0400 -- .../nifi/controller/ConfigurationContext.java | 23 +++-- .../nifi/util/MockConfigurationContext.java | 11 + .../apache/nifi/controller/FlowController.java | 18 +++--- .../reporting/AbstractReportingTaskNode.java| 4 +-- .../scheduling/StandardProcessScheduler.java| 24 +- .../service/StandardConfigurationContext.java | 26 +++- .../service/StandardControllerServiceNode.java | 2 +- .../StandardControllerServiceProvider.java | 10 .../apache/nifi/controller/MonitorMemory.java | 14 +++ 9 files changed, 89 insertions(+), 43 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java index 1fff6b9..03965d4 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java @@ -17,13 +17,14 @@ package org.apache.nifi.controller; import java.util.Map; +import java.util.concurrent.TimeUnit; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; /** - * This context is passed to ControllerServices after the service has been - * initialized. + * This context is passed to ControllerServices and Reporting Tasks in order + * to expose their configuration to them. */ public interface ConfigurationContext { @@ -39,4 +40,22 @@ public interface ConfigurationContext { */ MapPropertyDescriptor, String getProperties(); +/** + * @return a String representation of the scheduling period, or codenull/code if + * the component does not have a scheduling period (e.g., for ControllerServices) + */ +String getSchedulingPeriod(); + +/** + * Returns the amount of time, in the given {@link TimeUnit} that will + * elapsed between the return of one execution of the + * component's codeonTrigger/code method and + * the time at which the method is invoked again. This method will return + * null if the component does not have a scheduling period (e.g., for ControllerServices) + * + * @param timeUnit unit of time for scheduling + * @return period of time or codenull/code if component does not have a scheduling + * period + */ +Long getSchedulingPeriod(TimeUnit timeUnit); } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java -- diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java index 61af49d..e9afb32 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java @@ -18,6 +18,7 @@ package org.apache.nifi.util; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.TimeUnit; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; @@ -47,4 +48,14 @@ public class MockConfigurationContext implements ConfigurationContext { public MapPropertyDescriptor, String getProperties() { return new HashMap(this.properties); } + +@Override +public String getSchedulingPeriod() { +return 0 secs; +} + +@Override +public Long getSchedulingPeriod(final TimeUnit timeUnit) { +return 0L; +} }
[2/2] incubator-nifi git commit: NIFI-719: merged with develop
NIFI-719: merged with develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a0918079 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a0918079 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a0918079 Branch: refs/heads/develop Commit: a09180799d8a84108c3f5665668bded284768ecf Parents: 5021570 ba31559 Author: Mark Payne marka...@hotmail.com Authored: Wed Jul 1 10:44:42 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jul 1 10:44:42 2015 -0400 -- .../nifi/controller/ConfigurationContext.java | 23 +++-- .../nifi/util/MockConfigurationContext.java | 11 + .../mock/MockConfigurationContext.java | 10 .../apache/nifi/controller/FlowController.java | 18 +++--- .../reporting/AbstractReportingTaskNode.java| 4 +-- .../scheduling/StandardProcessScheduler.java| 24 +- .../service/StandardConfigurationContext.java | 26 +++- .../service/StandardControllerServiceNode.java | 2 +- .../StandardControllerServiceProvider.java | 10 .../apache/nifi/controller/MonitorMemory.java | 14 +++ 10 files changed, 99 insertions(+), 43 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a0918079/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java -- diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java index 742f03b,e9afb32..c90e722 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java @@@ -55,12 -49,13 +56,22 @@@ public class MockConfigurationContext i return new HashMap(this.properties); } +private PropertyDescriptor getActualDescriptor(final PropertyDescriptor property) { +if (service == null) { +return property; +} + +final PropertyDescriptor resolved = service.getPropertyDescriptor(property.getName()); +return resolved == null ? property : resolved; +} ++ + @Override + public String getSchedulingPeriod() { + return 0 secs; + } + + @Override + public Long getSchedulingPeriod(final TimeUnit timeUnit) { + return 0L; + } } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a0918079/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java -- diff --cc nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java index 1ba5499,000..6c9ec9d mode 100644,00..100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java @@@ -1,38 -1,0 +1,48 @@@ +/* + * 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.documentation.mock; + +import java.util.Collections; +import java.util.Map; ++import java.util.concurrent.TimeUnit; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.controller.ConfigurationContext; + +public class MockConfigurationContext implements ConfigurationContext { + +@Override +public PropertyValue getProperty(PropertyDescriptor property) { +return null; +} + +@Override +public MapPropertyDescriptor, String getProperties() { +return Collections.emptyMap(); +} + ++
incubator-nifi git commit: NIFI-731: Try using a Stack to hold FLowFIle Repo partitions so that we tend to use the same partition to avoid disk seeks/thrashing
Repository: incubator-nifi Updated Branches: refs/heads/NIFI-731 bb5128be2 - 4b5ba3c4e NIFI-731: Try using a Stack to hold FLowFIle Repo partitions so that we tend to use the same partition to avoid disk seeks/thrashing Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/4b5ba3c4 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/4b5ba3c4 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/4b5ba3c4 Branch: refs/heads/NIFI-731 Commit: 4b5ba3c4e9bbb2be5bd3f9bdf40159179a3b30e2 Parents: bb5128b Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 29 17:32:27 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 30 11:04:42 2015 -0400 -- .../org/wali/MinimalLockingWriteAheadLog.java | 173 +-- .../WriteAheadFlowFileRepository.java | 6 +- .../claim/StandardContentClaimManager.java | 19 +- 3 files changed, 132 insertions(+), 66 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4b5ba3c4/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java -- diff --git a/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java b/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java index 4331504..09887f7 100644 --- a/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java +++ b/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java @@ -43,6 +43,7 @@ import java.util.Queue; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; +import java.util.Stack; import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; @@ -82,11 +83,12 @@ public final class MinimalLockingWriteAheadLogT implements WriteAheadRepositor private final SortedSetPath basePaths; private final SerDeT serde; private final SyncListener syncListener; +private final int updatesBetweenSync; private final FileChannel lockChannel; private final AtomicLong transactionIdGenerator = new AtomicLong(0L); private final PartitionT[] partitions; -private final AtomicLong partitionIndex = new AtomicLong(0L); +private final StackPartitionT updatePartitionStack; private final ConcurrentMapObject, T recordMap = new ConcurrentHashMap(); private final MapObject, T unmodifiableRecordMap = Collections.unmodifiableMap(recordMap); private final SetString externalLocations = new CopyOnWriteArraySet(); @@ -108,6 +110,10 @@ public final class MinimalLockingWriteAheadLogT implements WriteAheadRepositor this(new TreeSet(Collections.singleton(path)), partitionCount, serde, syncListener); } +public MinimalLockingWriteAheadLog(final Path path, final int partitionCount, final SerDeT serde, final SyncListener syncListener, final int updatesBetweenSync) throws IOException { +this(new TreeSet(Collections.singleton(path)), partitionCount, serde, syncListener, updatesBetweenSync); +} + /** * * @param paths a sorted set of Paths to use for the partitions/journals and @@ -121,6 +127,25 @@ public final class MinimalLockingWriteAheadLogT implements WriteAheadRepositor * @throws IOException if unable to initialize due to IO issue */ public MinimalLockingWriteAheadLog(final SortedSetPath paths, final int partitionCount, final SerDeT serde, final SyncListener syncListener) throws IOException { +this(paths, partitionCount, serde, syncListener, 0); +} + +/** +* +* @param paths a sorted set of Paths to use for the partitions/journals and +* the snapshot. The snapshot will always be written to the first path +* specified. +* @param partitionCount the number of partitions/journals to use. For best +* performance, this should be close to the number of threads that are +* expected to update the repository simultaneously +* @param serde the serializer/deserializer for records +* @param syncListener the listener +* @param updatesBetweenSync if 0, each partition will be sync'ed after it is updated +* this number of times +* @throws IOException if unable to initialize due to IO issue +*/ +public MinimalLockingWriteAheadLog(final SortedSetPath paths, final int partitionCount, final SerDeT serde, final SyncListener syncListener, final int updatesBetweenSync) throws IOException { +this.updatesBetweenSync = updatesBetweenSync; this.syncListener = syncListener; this.basePaths = paths; @@ -161,6 +186,10 @@
[2/3] incubator-nifi git commit: Revert NIFI-731: Try using a Stack to hold FLowFIle Repo partitions so that we tend to use the same partition to avoid disk seeks/thrashing
Revert NIFI-731: Try using a Stack to hold FLowFIle Repo partitions so that we tend to use the same partition to avoid disk seeks/thrashing This reverts commit 4b5ba3c4e9bbb2be5bd3f9bdf40159179a3b30e2. Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/b3eb4a1a Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b3eb4a1a Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b3eb4a1a Branch: refs/heads/NIFI-731 Commit: b3eb4a1a0f29d0f94639174dfbd7ea05307cb99a Parents: 1677db6 Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 30 13:59:57 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 30 13:59:57 2015 -0400 -- .../org/wali/MinimalLockingWriteAheadLog.java | 173 ++- .../WriteAheadFlowFileRepository.java | 6 +- .../claim/StandardContentClaimManager.java | 19 +- 3 files changed, 66 insertions(+), 132 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b3eb4a1a/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java -- diff --git a/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java b/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java index 09887f7..4331504 100644 --- a/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java +++ b/nifi/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java @@ -43,7 +43,6 @@ import java.util.Queue; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; -import java.util.Stack; import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; @@ -83,12 +82,11 @@ public final class MinimalLockingWriteAheadLogT implements WriteAheadRepositor private final SortedSetPath basePaths; private final SerDeT serde; private final SyncListener syncListener; -private final int updatesBetweenSync; private final FileChannel lockChannel; private final AtomicLong transactionIdGenerator = new AtomicLong(0L); private final PartitionT[] partitions; -private final StackPartitionT updatePartitionStack; +private final AtomicLong partitionIndex = new AtomicLong(0L); private final ConcurrentMapObject, T recordMap = new ConcurrentHashMap(); private final MapObject, T unmodifiableRecordMap = Collections.unmodifiableMap(recordMap); private final SetString externalLocations = new CopyOnWriteArraySet(); @@ -110,10 +108,6 @@ public final class MinimalLockingWriteAheadLogT implements WriteAheadRepositor this(new TreeSet(Collections.singleton(path)), partitionCount, serde, syncListener); } -public MinimalLockingWriteAheadLog(final Path path, final int partitionCount, final SerDeT serde, final SyncListener syncListener, final int updatesBetweenSync) throws IOException { -this(new TreeSet(Collections.singleton(path)), partitionCount, serde, syncListener, updatesBetweenSync); -} - /** * * @param paths a sorted set of Paths to use for the partitions/journals and @@ -127,25 +121,6 @@ public final class MinimalLockingWriteAheadLogT implements WriteAheadRepositor * @throws IOException if unable to initialize due to IO issue */ public MinimalLockingWriteAheadLog(final SortedSetPath paths, final int partitionCount, final SerDeT serde, final SyncListener syncListener) throws IOException { -this(paths, partitionCount, serde, syncListener, 0); -} - -/** -* -* @param paths a sorted set of Paths to use for the partitions/journals and -* the snapshot. The snapshot will always be written to the first path -* specified. -* @param partitionCount the number of partitions/journals to use. For best -* performance, this should be close to the number of threads that are -* expected to update the repository simultaneously -* @param serde the serializer/deserializer for records -* @param syncListener the listener -* @param updatesBetweenSync if 0, each partition will be sync'ed after it is updated -* this number of times -* @throws IOException if unable to initialize due to IO issue -*/ -public MinimalLockingWriteAheadLog(final SortedSetPath paths, final int partitionCount, final SerDeT serde, final SyncListener syncListener, final int updatesBetweenSync) throws IOException { -this.updatesBetweenSync = updatesBetweenSync; this.syncListener = syncListener; this.basePaths = paths; @@ -186,10 +161,6 @@ public final class
[1/3] incubator-nifi git commit: NIFI-731: Deleted unused method
Repository: incubator-nifi Updated Branches: refs/heads/NIFI-731 4b5ba3c4e - 94b4eddb0 NIFI-731: Deleted unused method Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/1677db62 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1677db62 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1677db62 Branch: refs/heads/NIFI-731 Commit: 1677db6261c2c77df15c2e46a54d7428fc91fc57 Parents: 4b5ba3c Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 30 11:38:41 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 30 11:38:41 2015 -0400 -- .../repository/FileSystemRepository.java| 41 1 file changed, 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1677db62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index 864058c..c3f03db 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java @@ -805,47 +805,6 @@ public class FileSystemRepository implements ContentRepository { contentClaimManager.purge(); } -// private class BinDestructableClaims implements Runnable { -// -// @Override -// public void run() { -// try { -// // Get all of the Destructable Claims and bin them based on their Container. We do this -// // because the Container generally maps to a physical partition on the disk, so we want a few -// // different threads hitting the different partitions but don't want multiple threads hitting -// // the same partition. -// final ListContentClaim toDestroy = new ArrayList(); -// while (true) { -// toDestroy.clear(); -// contentClaimManager.drainDestructableClaims(toDestroy, 1); -// if (toDestroy.isEmpty()) { -// return; -// } -// -// for (final ContentClaim claim : toDestroy) { -// final String container = claim.getContainer(); -// final BlockingQueueContentClaim claimQueue = reclaimable.get(container); -// -// try { -// while (true) { -// if (claimQueue.offer(claim, 10, TimeUnit.MINUTES)) { -// break; -// } else { -// LOG.warn(Failed to clean up {} because old claims aren't being cleaned up fast enough. -// + This Content Claim will remain in the Content Repository until NiFi is restarted, at which point it will be cleaned up, claim); -// } -// } -// } catch (final InterruptedException ie) { -// LOG.warn(Failed to clean up {} because thread was interrupted, claim); -// } -// } -// } -// } catch (final Throwable t) { -// LOG.error(Failed to cleanup content claims due to {}, t); -// } -// } -// } - public static Path getArchivePath(final Path contentClaimPath) { final Path sectionPath = contentClaimPath.getParent(); final String claimId = contentClaimPath.toFile().getName();
[3/3] incubator-nifi git commit: NIFI-731: fixed issue with how bulletins were emitted by content claim manager
NIFI-731: fixed issue with how bulletins were emitted by content claim manager Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/94b4eddb Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/94b4eddb Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/94b4eddb Branch: refs/heads/NIFI-731 Commit: 94b4eddb00055fccbe57c5e3fb2ed93c0e91c153 Parents: b3eb4a1 Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 30 14:09:56 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 30 14:09:56 2015 -0400 -- .../repository/claim/StandardContentClaimManager.java | 10 ++ 1 file changed, 2 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94b4eddb/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaimManager.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaimManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaimManager.java index f5494f9..68ec5d9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaimManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaimManager.java @@ -123,17 +123,11 @@ public class StandardContentClaimManager implements ContentClaimManager { final BlockingQueueContentClaim destructableQueue = getDestructableClaimQueue(claim.getContainer()); final boolean accepted = destructableQueue.offer(claim); if (!accepted) { -final long start = System.nanoTime(); - while (!destructableQueue.offer(claim, 30, TimeUnit.MINUTES)) { } -final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); -if (millis 10L) { -logger.warn(Total wait duration to add claim to Destructable Claim Queue was {} millis, millis); -eventReporter.reportEvent(Severity.WARNING, Content Repository, The Content Repository is unable to destroy content as fast -+ as it is being created. The flow will be slowed in order to adjust for this.); -} +eventReporter.reportEvent(Severity.WARNING, Content Repository, The Content Repository is unable to destroy content as fast ++ as it is being created. The flow will be slowed in order to adjust for this.); } } catch (final InterruptedException ie) { }
[2/2] incubator-nifi git commit: NIFI-731: Updated admin guide to explain the flowfile repo properties more; added new property for now often to 'sync' the flowfile repo; allowed number of partitions
NIFI-731: Updated admin guide to explain the flowfile repo properties more; added new property for now often to 'sync' the flowfile repo; allowed number of partitions in flowfile repo to be changed after repo created. Minor tweaks to content repo and its communication with flowfile repo via ContentClaimManager Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/bb5128be Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/bb5128be Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/bb5128be Branch: refs/heads/NIFI-731 Commit: bb5128be2246dc48d5200e98572c47ff7cdfb905 Parents: fb0a1a7 Author: Mark Payne marka...@hotmail.com Authored: Sun Jun 28 13:07:04 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sun Jun 28 13:36:43 2015 -0400 -- nifi/nifi-assembly/pom.xml | 3 +- .../org/apache/nifi/util/NiFiProperties.java| 63 +++-- .../org/wali/MinimalLockingWriteAheadLog.java | 99 +++- .../wali/TestMinimalLockingWriteAheadLog.java | 97 ++- .../src/main/asciidoc/administration-guide.adoc | 25 - .../apache/nifi/controller/FlowController.java | 25 ++--- .../repository/FileSystemRepository.java| 2 +- .../WriteAheadFlowFileRepository.java | 18 +++- .../claim/StandardContentClaimManager.java | 21 - .../repository/TestFileSystemRepository.java| 12 ++- .../repository/TestStandardProcessSession.java | 43 - .../TestVolatileContentRepository.java | 5 +- .../TestWriteAheadFlowFileRepository.java | 7 +- .../src/main/resources/conf/nifi.properties | 1 + 14 files changed, 311 insertions(+), 110 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/bb5128be/nifi/nifi-assembly/pom.xml -- diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index 9b17617..ac39ca6 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -229,9 +229,10 @@ language governing permissions and limitations under the License. -- nifi.flowfile.repository.implementationorg.apache.nifi.controller.repository.WriteAheadFlowFileRepository/nifi.flowfile.repository.implementation nifi.flowfile.repository.directory./flowfile_repository/nifi.flowfile.repository.directory - nifi.flowfile.repository.partitions256/nifi.flowfile.repository.partitions + nifi.flowfile.repository.partitions16/nifi.flowfile.repository.partitions nifi.flowfile.repository.checkpoint.interval2 mins/nifi.flowfile.repository.checkpoint.interval nifi.flowfile.repository.always.syncfalse/nifi.flowfile.repository.always.sync + nifi.flowfile.repository.updates.between.sync128/nifi.flowfile.repository.updates.between.sync nifi.swap.manager.implementationorg.apache.nifi.controller.FileSystemSwapManager/nifi.swap.manager.implementation nifi.queue.swap.threshold2/nifi.queue.swap.threshold nifi.swap.in.period5 sec/nifi.swap.in.period http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/bb5128be/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java -- diff --git a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index e25f5d6..6dadf12 100644 --- a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -80,7 +80,8 @@ public class NiFiProperties extends Properties { // flowfile repository properties public static final String FLOWFILE_REPOSITORY_IMPLEMENTATION = nifi.flowfile.repository.implementation; -public static final String FLOWFILE_REPOSITORY_ALWAYS_SYNC = nifi.flowfile.repository.always.sync; +public static final String FLOWFILE_REPOSITORY_ALWAYS_SYNC = nifi.flowfile.repository.always.sync; // deprecated +public static final String FLOWFILE_REPOSITORY_UPDATES_BETWEEN_SYNCS = nifi.flowfile.repository.updates.between.sync; public static final String FLOWFILE_REPOSITORY_DIRECTORY = nifi.flowfile.repository.directory; public static final String FLOWFILE_REPOSITORY_PARTITIONS = nifi.flowfile.repository.partitions; public static final String FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL = nifi.flowfile.repository.checkpoint.interval; @@ -282,7 +283,7 @@ public class NiFiProperties extends Properties { public File getFlowConfigurationFile() { try {
[1/2] incubator-nifi git commit: NIFI-731: Merged develop to branch
Repository: incubator-nifi Updated Branches: refs/heads/NIFI-731 [created] bb5128be2 NIFI-731: Merged develop to branch Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/fb0a1a76 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/fb0a1a76 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/fb0a1a76 Branch: refs/heads/NIFI-731 Commit: fb0a1a76f431148fa9664e565c94cf334498dbf8 Parents: e35f348 Author: Mark Payne marka...@hotmail.com Authored: Sun Jun 28 13:36:34 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sun Jun 28 13:36:34 2015 -0400 -- .../repository/claim/ContentClaimManager.java | 15 +- .../repository/FileSystemRepository.java| 230 ++- .../repository/VolatileContentRepository.java | 2 +- .../WriteAheadFlowFileRepository.java | 22 +- .../claim/StandardContentClaimManager.java | 36 ++- .../controller/TestFileSystemSwapManager.java | 4 +- 6 files changed, 172 insertions(+), 137 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fb0a1a76/nifi/nifi-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimManager.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimManager.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimManager.java index bffcec3..bf6acc0 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimManager.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimManager.java @@ -110,29 +110,30 @@ public interface ContentClaimManager { /** * Drains up to {@code maxElements} Content Claims from the internal queue - * of destructable content claims to the given {@code destination} so that - * they can be destroyed. + * of destructable content claims that belong to the given Content Repository container + * to the given {@code destination} so that they can be destroyed. * + * @param container the container to drain * @param destination to drain to * @param maxElements max items to drain */ -void drainDestructableClaims(CollectionContentClaim destination, int maxElements); +void drainDestructableClaims(String container, CollectionContentClaim destination, int maxElements); /** - * Drains up to {@code maxElements} Content Claims from the internal queue - * of destructable content claims to the given {@code destination} so that - * they can be destroyed. If no ContentClaim is ready to be destroyed at + * Drains up to {@code maxElements} Content Claims that belong to the given Content Repository + * container from the internal queue of destructable content claims to the given {@code destination} so that they can be destroyed. If no ContentClaim is ready to be destroyed at * this time, will wait up to the specified amount of time before returning. * If, after the specified amount of time, there is still no ContentClaim * ready to be destroyed, the method will return without having added * anything to the given {@code destination}. * + * @param container the container to drain * @param destination to drain to * @param maxElements max items to drain * @param timeout maximum time to wait * @param unit unit of time to wait */ -void drainDestructableClaims(CollectionContentClaim destination, int maxElements, long timeout, TimeUnit unit); +void drainDestructableClaims(String container, CollectionContentClaim destination, int maxElements, long timeout, TimeUnit unit); /** * Clears the manager's memory of any and all ContentClaims that it knows http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fb0a1a76/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index 1171636..4e4609a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++
incubator-nifi git commit: NIFI-728: Allow Mock Framework to use property descriptors from subclasses that are created for unit testing
Repository: incubator-nifi Updated Branches: refs/heads/develop 69297a3aa - f58972e56 NIFI-728: Allow Mock Framework to use property descriptors from subclasses that are created for unit testing Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f58972e5 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f58972e5 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f58972e5 Branch: refs/heads/develop Commit: f58972e566448886cdf984e6c8a34c83674b0705 Parents: 69297a3 Author: Mark Payne marka...@hotmail.com Authored: Thu Jun 25 09:46:16 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 25 09:56:07 2015 -0400 -- .../nifi/util/MockConfigurationContext.java | 18 +- .../nifi/util/StandardProcessorTestRunner.java| 4 ++-- 2 files changed, 19 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f58972e5/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java -- diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java index 61af49d..742f03b 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java @@ -22,14 +22,21 @@ import java.util.Map; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; public class MockConfigurationContext implements ConfigurationContext { private final MapPropertyDescriptor, String properties; private final ControllerServiceLookup serviceLookup; +private final ControllerService service; public MockConfigurationContext(final MapPropertyDescriptor, String properties, final ControllerServiceLookup serviceLookup) { +this(null, properties, serviceLookup); +} + +public MockConfigurationContext(final ControllerService service, final MapPropertyDescriptor, String properties, final ControllerServiceLookup serviceLookup) { +this.service = service; this.properties = properties; this.serviceLookup = serviceLookup; } @@ -38,7 +45,7 @@ public class MockConfigurationContext implements ConfigurationContext { public PropertyValue getProperty(final PropertyDescriptor property) { String value = properties.get(property); if (value == null) { -value = property.getDefaultValue(); +value = getActualDescriptor(property).getDefaultValue(); } return new MockPropertyValue(value, serviceLookup); } @@ -47,4 +54,13 @@ public class MockConfigurationContext implements ConfigurationContext { public MapPropertyDescriptor, String getProperties() { return new HashMap(this.properties); } + +private PropertyDescriptor getActualDescriptor(final PropertyDescriptor property) { +if (service == null) { +return property; +} + +final PropertyDescriptor resolved = service.getPropertyDescriptor(property.getName()); +return resolved == null ? property : resolved; +} } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f58972e5/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java -- diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 8938547..048e2b9 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -215,7 +215,7 @@ public class StandardProcessorTestRunner implements TestRunner { executorService.shutdown(); try { executorService.awaitTermination(runWait, TimeUnit.MILLISECONDS); -} catch (InterruptedException e1) { +} catch (final InterruptedException e1) { } int finishedCount = 0; @@ -609,7 +609,7 @@ public class StandardProcessorTestRunner implements TestRunner { } try { -final ConfigurationContext configContext = new MockConfigurationContext(configuration.getProperties(), context); +final ConfigurationContext
[1/2] incubator-nifi git commit: NIFI-724: Enable bulletins for reporting tasks and controller services
Repository: incubator-nifi Updated Branches: refs/heads/NIFI-724 f1adb8bf0 - 0a7ab1a06 NIFI-724: Enable bulletins for reporting tasks and controller services Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/e240e07a Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/e240e07a Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/e240e07a Branch: refs/heads/NIFI-724 Commit: e240e07aaebea1fd66b22fce8aec3f0005fd3f60 Parents: e767f5c Author: Mark Payne marka...@hotmail.com Authored: Wed Jun 24 14:03:34 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 25 10:56:49 2015 -0400 -- .../org/apache/nifi/reporting/Bulletin.java | 11 ++- .../apache/nifi/reporting/ComponentType.java| 58 ++ .../manager/impl/ClusteredReportingContext.java | 46 ++- .../cluster/manager/impl/WebClusterManager.java | 80 .../org/apache/nifi/events/BulletinFactory.java | 30 ++-- .../org/apache/nifi/events/SystemBulletin.java | 2 + .../nifi/logging/LogRepositoryFactory.java | 6 +- .../apache/nifi/controller/FlowController.java | 35 ++--- .../service/ControllerServiceLoader.java| 11 +-- .../nifi/events/VolatileBulletinRepository.java | 18 +++-- .../org/apache/nifi/jaxb/AdaptedBulletin.java | 10 +++ .../org/apache/nifi/jaxb/BulletinAdapter.java | 3 +- .../logging/ControllerServiceLogObserver.java | 45 +++ .../nifi/logging/ReportingTaskLogObserver.java | 45 +++ .../nifi/remote/StandardRemoteProcessGroup.java | 16 ++-- .../nifi/remote/StandardRootGroupPort.java | 4 +- 16 files changed, 349 insertions(+), 71 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e240e07a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java index 87443a3..fe370ae 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java @@ -34,6 +34,7 @@ public abstract class Bulletin implements ComparableBulletin { private String groupId; private String sourceId; private String sourceName; +private ComponentType sourceType; protected Bulletin(final long id) { this.timestamp = new Date(); @@ -104,9 +105,17 @@ public abstract class Bulletin implements ComparableBulletin { this.sourceName = sourceName; } +public ComponentType getSourceType() { +return sourceType; +} + +public void setSourceType(ComponentType sourceType) { +this.sourceType = sourceType; +} + @Override public String toString() { -return Bulletin{ + id= + id + , message= + message + , sourceName= + sourceName + '}'; +return Bulletin{ + id= + id + , message= + message + , sourceName= + sourceName + , sourceType= + sourceType + '}'; } @Override http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e240e07a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java new file mode 100644 index 000..97f3538 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java @@ -0,0 +1,58 @@ +/* + * 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.reporting; + +/** + * An Enumeration for indicating which type of component a Bulletin is associated with + */ +public enum ComponentType { + +/** + * Bulletin is associated with a Processor + */ +PROCESSOR, + +/** + * Bulletin is associated with a
[2/2] incubator-nifi git commit: NIFI-724: merged changes
NIFI-724: merged changes Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/0a7ab1a0 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/0a7ab1a0 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/0a7ab1a0 Branch: refs/heads/NIFI-724 Commit: 0a7ab1a06dd8b7164413bdcd0a85a4c457d3da71 Parents: e240e07 f1adb8b Author: Mark Payne marka...@hotmail.com Authored: Thu Jun 25 10:58:01 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 25 10:58:01 2015 -0400 -- --
incubator-nifi git commit: NIFI-724: Enable bulletins for reporting tasks and controller services
Repository: incubator-nifi Updated Branches: refs/heads/NIFI-724 [created] f1adb8bf0 NIFI-724: Enable bulletins for reporting tasks and controller services Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f1adb8bf Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f1adb8bf Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f1adb8bf Branch: refs/heads/NIFI-724 Commit: f1adb8bf034a8407dfdd655e441c74c10a61b18f Parents: e767f5c Author: Mark Payne marka...@hotmail.com Authored: Wed Jun 24 14:03:34 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 24 14:03:34 2015 -0400 -- .../org/apache/nifi/reporting/Bulletin.java | 11 ++- .../apache/nifi/reporting/ComponentType.java| 58 ++ .../manager/impl/ClusteredReportingContext.java | 46 ++- .../cluster/manager/impl/WebClusterManager.java | 80 .../org/apache/nifi/events/BulletinFactory.java | 30 ++-- .../org/apache/nifi/events/SystemBulletin.java | 2 + .../nifi/logging/LogRepositoryFactory.java | 6 +- .../apache/nifi/controller/FlowController.java | 35 ++--- .../service/ControllerServiceLoader.java| 11 +-- .../nifi/events/VolatileBulletinRepository.java | 7 +- .../org/apache/nifi/jaxb/AdaptedBulletin.java | 10 +++ .../org/apache/nifi/jaxb/BulletinAdapter.java | 3 +- .../logging/ControllerServiceLogObserver.java | 45 +++ .../nifi/logging/ReportingTaskLogObserver.java | 45 +++ .../nifi/remote/StandardRemoteProcessGroup.java | 16 ++-- .../nifi/remote/StandardRootGroupPort.java | 4 +- 16 files changed, 341 insertions(+), 68 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f1adb8bf/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java index 87443a3..fe370ae 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java @@ -34,6 +34,7 @@ public abstract class Bulletin implements ComparableBulletin { private String groupId; private String sourceId; private String sourceName; +private ComponentType sourceType; protected Bulletin(final long id) { this.timestamp = new Date(); @@ -104,9 +105,17 @@ public abstract class Bulletin implements ComparableBulletin { this.sourceName = sourceName; } +public ComponentType getSourceType() { +return sourceType; +} + +public void setSourceType(ComponentType sourceType) { +this.sourceType = sourceType; +} + @Override public String toString() { -return Bulletin{ + id= + id + , message= + message + , sourceName= + sourceName + '}'; +return Bulletin{ + id= + id + , message= + message + , sourceName= + sourceName + , sourceType= + sourceType + '}'; } @Override http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f1adb8bf/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java new file mode 100644 index 000..97f3538 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ComponentType.java @@ -0,0 +1,58 @@ +/* + * 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.reporting; + +/** + * An Enumeration for indicating which type of component a Bulletin is associated with + */ +public enum ComponentType { + +/** + * Bulletin is associated with a Processor + */ +PROCESSOR, + +/** + * Bulletin is associated with a Remote
[1/2] incubator-nifi git commit: This closes #60
Repository: incubator-nifi Updated Branches: refs/heads/develop b38d74a65 - a6de1b425 This closes #60 Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/c45dcf0b Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c45dcf0b Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c45dcf0b Branch: refs/heads/develop Commit: c45dcf0b52b6a856bc238a54adf45addc78651e8 Parents: 25146a5 Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 23 10:24:58 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 23 10:24:58 2015 -0400 -- --
[2/2] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a6de1b42 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a6de1b42 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a6de1b42 Branch: refs/heads/develop Commit: a6de1b42504471f9f99aca8f8fe78f1cba85c8e1 Parents: c45dcf0 b38d74a Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 23 10:29:11 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 23 10:29:11 2015 -0400 -- .../nifi/provenance/lineage/LineageNode.java| 1 + .../nifi/provenance/lineage/EventNode.java | 2 ++ .../nifi/provenance/lineage/FlowFileNode.java | 1 + .../org/apache/nifi/util/NiFiProperties.java| 22 4 files changed, 17 insertions(+), 9 deletions(-) --
[1/2] incubator-nifi git commit: NIFI-378: updated documentation to explain contract of MergeContent's Defragment strategy a bit more clearly
Repository: incubator-nifi Updated Branches: refs/heads/develop 169256015 - e767f5ce0 NIFI-378: updated documentation to explain contract of MergeContent's Defragment strategy a bit more clearly Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/ddad70ba Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/ddad70ba Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/ddad70ba Branch: refs/heads/develop Commit: ddad70ba009a875e088020ae1ea8f29caeb529f9 Parents: 5d8bfa7 Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 22 12:42:22 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 12:42:22 2015 -0400 -- .../nifi/processors/standard/MergeContent.java | 57 +++- 1 file changed, 31 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ddad70ba/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index 2883a75..65f4124 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -82,31 +82,34 @@ import org.apache.nifi.util.ObjectHolder; + created from FlowFiles in different connections. This processor updates the mime.type attribute as appropriate.) @ReadsAttributes({ @ReadsAttribute(attribute = fragment.identifier, description = Applicable only if the Merge Strategy property is set to Defragment. -+ All FlowFiles with the same value for this attribute will be bundled together), -@ReadsAttribute(attribute = fragment.index, description = Applicable only if the Merge Strategy property is set to Defragment. This -+ attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute and must be a unique integer -+ between 0 and the value of the fragment.count attribute. This attribute indicates the order in which the fragments should be assembled), ++ All FlowFiles with the same value for this attribute will be bundled together.), +@ReadsAttribute(attribute = fragment.index, description = Applicable only if the Merge Strategy property is set to Defragment. ++ This attribute indicates the order in which the fragments should be assembled. This ++ attribute must be present on all FlowFiles when using the Defragment Merge Strategy and must be a unique (i.e., unique across all ++ FlowFiles that have the same value for the \fragment.identifier\ attribute) integer ++ between 0 and the value of the fragment.count attribute. If two or more FlowFiles have the same value for the ++ \fragment.identifier\ attribute and the same value for the \fragment.index\ attribute, the behavior of this Processor is undefined.), @ReadsAttribute(attribute = fragment.count, description = Applicable only if the Merge Strategy property is set to Defragment. This -+ attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All FlowFiles in the same -+ bundle must have the same value for this attribute. The value of this attribute indicates how many FlowFiles should be expected -+ in the given bundle), ++ attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All FlowFiles in the same ++ bundle must have the same value for this attribute. The value of this attribute indicates how many FlowFiles should be expected ++ in the given bundle.), @ReadsAttribute(attribute = segment.original.filename, description = Applicable only if the Merge Strategy property is set to Defragment. -+ This attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All FlowFiles in the same -+ bundle must have the same value for this attribute. The value of this attribute will be used for the filename of the completed merged -+ FlowFile), ++ This attribute must be present on all FlowFiles with the same value for the fragment.identifier attribute. All
[2/2] incubator-nifi git commit: Merge branch 'NIFI-378' into develop
Merge branch 'NIFI-378' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/e767f5ce Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/e767f5ce Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/e767f5ce Branch: refs/heads/develop Commit: e767f5ce02c36f41874d4f97abdb50c643662deb Parents: 1692560 ddad70b Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 23 11:17:30 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 23 11:17:30 2015 -0400 -- .../nifi/processors/standard/MergeContent.java | 57 +++- 1 file changed, 31 insertions(+), 26 deletions(-) --
[2/2] incubator-nifi git commit: Merge branch 'NIFI-718' into develop
Merge branch 'NIFI-718' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/16925601 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/16925601 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/16925601 Branch: refs/heads/develop Commit: 169256015bdef5d2bae6d4676ef776b06cdb42cd Parents: a6de1b4 1faca8c Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 23 11:03:47 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 23 11:03:47 2015 -0400 -- .../nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh | 4 1 file changed, 4 insertions(+) --
[1/2] incubator-nifi git commit: NIFI-718: Add links to /etc/rc2.d when installing nifi as a linux service
Repository: incubator-nifi Updated Branches: refs/heads/develop a6de1b425 - 169256015 NIFI-718: Add links to /etc/rc2.d when installing nifi as a linux service Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/1faca8c9 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1faca8c9 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1faca8c9 Branch: refs/heads/develop Commit: 1faca8c93ea617a80bbeaa7ebd318b477f4ebd07 Parents: 25146a5 Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 23 09:53:40 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 23 09:53:40 2015 -0400 -- .../nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh | 4 1 file changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1faca8c9/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh index 6d3191d..8caf55e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi.sh @@ -140,6 +140,10 @@ install() { cp $0 $SVC_FILE sed -i s:NIFI_HOME=.*:NIFI_HOME=$NIFI_HOME: $SVC_FILE sed -i s:PROGNAME=.*:PROGNAME=$(basename $0): $SVC_FILE +rm -f /etc/rc2.d/S65${SVC_NAME} +ln -s /etc/init.d/$SVC_NAME /etc/rc2.d/S65${SVC_NAME} +rm -f /etc/rc2.d/K65${SVC_NAME} +ln -s /etc/init.d/$SVC_NAME /etc/rc2.d/K65${SVC_NAME} echo Service $SVC_NAME installed }
incubator-nifi git commit: NIFI-704 StandardProcessorTestRunner should allow you to wait before calling OnUnScheduled methods
Repository: incubator-nifi Updated Branches: refs/heads/develop f2f905605 - 5d8bfa7c8 NIFI-704 StandardProcessorTestRunner should allow you to wait before calling OnUnScheduled methods Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5d8bfa7c Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5d8bfa7c Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5d8bfa7c Branch: refs/heads/develop Commit: 5d8bfa7c806549f5ede6eec5047dc66696bc95d8 Parents: f2f9056 Author: danbress dbr...@onyxconsults.com Authored: Sat Jun 20 19:53:13 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 12:08:41 2015 -0400 -- .../nifi/util/StandardProcessorTestRunner.java | 10 +++ .../java/org/apache/nifi/util/TestRunner.java | 45 + .../CurrentTestStandardProcessorTestRunner.java | 71 3 files changed, 126 insertions(+) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5d8bfa7c/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java -- diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 655f2df..8938547 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -39,6 +39,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -182,6 +183,11 @@ public class StandardProcessorTestRunner implements TestRunner { @Override public void run(final int iterations, final boolean stopOnFinish, final boolean initialize) { +run(iterations, stopOnFinish, initialize, 5000); +} + +@Override +public void run(final int iterations, final boolean stopOnFinish, final boolean initialize, final long runWait) { if (iterations 1) { throw new IllegalArgumentException(); } @@ -207,6 +213,10 @@ public class StandardProcessorTestRunner implements TestRunner { } executorService.shutdown(); +try { +executorService.awaitTermination(runWait, TimeUnit.MILLISECONDS); +} catch (InterruptedException e1) { +} int finishedCount = 0; boolean unscheduledRun = false; http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5d8bfa7c/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java -- diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java index a599e5b..fb9fc78 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java @@ -126,6 +126,51 @@ public interface TestRunner { void run(int iterations, boolean stopOnFinish, final boolean initialize); /** + * This method runs the {@link Processor} codeiterations/code times, + * using the sequence of steps below: + * ul + * li + * If {@code initialize} is true, run all methods on the Processor that are + * annotated with the + * {@link nifi.processor.annotation.OnScheduled @OnScheduled} annotation. If + * any of these methods throws an Exception, the Unit Test will fail. + * /li + * li + * Schedule the + * {@link Processor#onTrigger(ProcessContext, ProcessSessionFactory) onTrigger} + * method to be invoked codeiterations/code times. The number of threads + * used to run these iterations is determined by the ThreadCount of this + * codeTestRunner/code. By default, the value is set to 1, but it can be + * modified by calling the {@link #setThreadCount(int)} method. + * /li + * li + * As soon as the first thread finishes its execution of + * {@link Processor#onTrigger(ProcessContext, ProcessSessionFactory) onTrigger}, + * all methods on the Processor that are annotated with the + * {@link nifi.processor.annotation.OnUnscheduled @OnUnscheduled} annotation + * are invoked. If any of these methods throws an Exception, the Unit Test + * will fail. + * /li + * li + * Waits for all threads to finish
incubator-nifi git commit: NIFI-711: Do not check status of FlowFile when emitting provenance CLONE event during session commit/checkpoint
Repository: incubator-nifi Updated Branches: refs/heads/develop 77a0561bc - b22a1261c NIFI-711: Do not check status of FlowFile when emitting provenance CLONE event during session commit/checkpoint Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/b22a1261 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b22a1261 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b22a1261 Branch: refs/heads/develop Commit: b22a1261c3f29c8714d043174b7beff36ebe5549 Parents: 77a0561 Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 22 11:31:55 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 13:30:53 2015 -0400 -- .../nifi/controller/repository/StandardProcessSession.java | 2 +- .../controller/repository/StandardProvenanceReporter.java| 8 +++- 2 files changed, 8 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b22a1261/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 4ee8c06..04e819e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -260,7 +260,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE final FlowFileRecord clone = builder.build(); final StandardRepositoryRecord newRecord = new StandardRepositoryRecord(destination.getFlowFileQueue()); -getProvenanceReporter().clone(currRec, clone); +provenanceReporter.clone(currRec, clone, false); final ContentClaim claim = clone.getContentClaim(); if (claim != null) { http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b22a1261/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java index 5194fef..8852f42 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java @@ -327,7 +327,13 @@ public class StandardProvenanceReporter implements ProvenanceReporter { @Override public void clone(final FlowFile parent, final FlowFile child) { -verifyFlowFileKnown(child); +clone(parent, child, true); +} + +void clone(final FlowFile parent, final FlowFile child, final boolean verifyFlowFile) { +if (verifyFlowFile) { +verifyFlowFileKnown(child); +} try { final ProvenanceEventBuilder eventBuilder = build(parent, ProvenanceEventType.CLONE);
incubator-nifi git commit: NIFI-672: Fixed typo in Admin Guide
Repository: incubator-nifi Updated Branches: refs/heads/develop 5d8bfa7c8 - 77a0561bc NIFI-672: Fixed typo in Admin Guide Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/77a0561b Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/77a0561b Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/77a0561b Branch: refs/heads/develop Commit: 77a0561bc059b7503867d583245b025d08244dc8 Parents: 5d8bfa7 Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 22 12:50:17 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 12:50:17 2015 -0400 -- nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/77a0561b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc -- diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc index 039e5b9..5535c35 100644 --- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -210,7 +210,7 @@ Here is an example entry using the name John Smith: users -user dn=[cn=John Smith,ou=people,dc=example,dc=com] +user dn=cn=John Smith,ou=people,dc=example,dc=com role name=ROLE_ADMIN/ /user /users
[01/10] incubator-nifi git commit: Merge pull request #1 from apache/develop
Repository: incubator-nifi Updated Branches: refs/heads/develop c4c98fc4a - f2f905605 Merge pull request #1 from apache/develop Merging latest NiFi to my fork Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/29b4e56a Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/29b4e56a Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/29b4e56a Branch: refs/heads/develop Commit: 29b4e56a61dc7125e10d7e5d70a0123485717b4d Parents: e7d6d94 c026dff Author: Brian Ghigiarelli briang...@gmail.com Authored: Wed Apr 22 11:32:36 2015 -0400 Committer: Brian Ghigiarelli briang...@gmail.com Committed: Wed Apr 22 11:32:36 2015 -0400 -- .../nifi/annotation/lifecycle/OnAdded.java | 13 +- .../nifi/annotation/lifecycle/OnDisabled.java | 29 +- .../nifi/annotation/lifecycle/OnEnabled.java| 34 +- .../nifi/annotation/lifecycle/OnRemoved.java| 14 +- .../nifi/annotation/lifecycle/OnShutdown.java | 13 +- .../nifi/annotation/lifecycle/OnStopped.java|9 + .../annotation/lifecycle/OnUnscheduled.java |2 - .../nifi/components/PropertyDescriptor.java | 14 +- .../nifi/components/ValidationContext.java |9 + .../controller/AbstractControllerService.java | 13 +- .../ControllerServiceInitializationContext.java | 10 + .../controller/ControllerServiceLookup.java | 19 + .../org/apache/nifi/logging/ComponentLog.java | 100 + .../org/apache/nifi/logging/ProcessorLog.java | 61 +- .../nifi/reporting/AbstractReportingTask.java | 10 + .../ReportingInitializationContext.java | 10 + .../nifi/web/ClusterRequestException.java |1 + .../org/apache/nifi/web/ComponentDetails.java | 157 ++ .../apache/nifi/web/ConfigurationAction.java| 137 ++ .../nifi/web/NiFiWebConfigurationContext.java | 102 + .../web/NiFiWebConfigurationRequestContext.java | 31 + .../org/apache/nifi/web/NiFiWebContext.java |1 + .../apache/nifi/web/NiFiWebContextConfig.java |1 + .../apache/nifi/web/NiFiWebRequestContext.java | 58 + .../nifi/web/ProcessorConfigurationAction.java |1 + .../java/org/apache/nifi/web/ProcessorInfo.java |1 + .../main/java/org/apache/nifi/web/Revision.java | 32 +- .../org/apache/nifi/web/UiExtensionType.java| 31 + nifi/nifi-assembly/NOTICE | 65 + nifi/nifi-assembly/pom.xml | 927 +- .../nifi-hl7-query-language/.gitignore |3 + .../nifi-hl7-query-language/pom.xml | 122 ++ .../apache/nifi/hl7/query/antlr/HL7QueryLexer.g | 173 ++ .../nifi/hl7/query/antlr/HL7QueryParser.g | 108 ++ .../org/apache/nifi/hl7/hapi/EmptyField.java| 37 + .../org/apache/nifi/hl7/hapi/HapiField.java | 83 + .../org/apache/nifi/hl7/hapi/HapiMessage.java | 94 + .../org/apache/nifi/hl7/hapi/HapiSegment.java | 69 + .../apache/nifi/hl7/hapi/SingleValueField.java | 42 + .../java/org/apache/nifi/hl7/io/HL7Reader.java | 27 + .../hl7/io/exception/InvalidHL7Exception.java | 40 + .../org/apache/nifi/hl7/model/HL7Component.java | 24 + .../org/apache/nifi/hl7/model/HL7Field.java | 21 + .../org/apache/nifi/hl7/model/HL7Message.java | 27 + .../org/apache/nifi/hl7/model/HL7Segment.java | 27 + .../org/apache/nifi/hl7/query/Declaration.java | 29 + .../org/apache/nifi/hl7/query/HL7Query.java | 412 + .../org/apache/nifi/hl7/query/QueryResult.java | 29 + .../org/apache/nifi/hl7/query/ResultHit.java| 25 + .../org/apache/nifi/hl7/query/Selection.java| 37 + .../hl7/query/evaluator/BooleanEvaluator.java | 24 + .../nifi/hl7/query/evaluator/Evaluator.java | 27 + .../hl7/query/evaluator/IntegerEvaluator.java | 26 + .../hl7/query/evaluator/StringEvaluator.java| 25 + .../comparison/AbstractComparisonEvaluator.java | 106 ++ .../comparison/AbstractNumericComparison.java | 67 + .../evaluator/comparison/EqualsEvaluator.java | 32 + .../comparison/GreaterThanEvaluator.java| 34 + .../comparison/GreaterThanOrEqualEvaluator.java | 34 + .../evaluator/comparison/IsNullEvaluator.java | 69 + .../evaluator/comparison/LessThanEvaluator.java | 31 + .../comparison/LessThanOrEqualEvaluator.java| 31 + .../comparison/NotEqualsEvaluator.java | 32 + .../evaluator/comparison/NotEvaluator.java | 36 + .../evaluator/comparison/NotNullEvaluator.java | 65 + .../literal/IntegerLiteralEvaluator.java| 36 + .../literal/StringLiteralEvaluator.java | 35 + .../hl7/query/evaluator/logic/AndEvaluator.java | 43 + .../hl7/query/evaluator/logic/OrEvaluator.java | 43 + .../message/DeclaredReferenceEvaluator.java | 42 + .../query/evaluator/message/DotEvaluator.java | 88 +
[02/10] incubator-nifi git commit: Merge pull request #2 from apache/develop
Merge pull request #2 from apache/develop Merging latest Apache NiFi to fork Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/421ad8fb Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/421ad8fb Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/421ad8fb Branch: refs/heads/develop Commit: 421ad8fb133d3bab32bc20d98011e9dfa0caff99 Parents: 29b4e56 e2750d4 Author: Brian Ghigiarelli briang...@gmail.com Authored: Wed May 13 20:32:23 2015 -0400 Committer: Brian Ghigiarelli briang...@gmail.com Committed: Wed May 13 20:32:23 2015 -0400 -- KEYS| 58 + README.md | 39 +- nifi-nar-maven-plugin/README.md | 18 +- nifi-nar-maven-plugin/pom.xml | 184 +- .../src/main/java/org/apache/nifi/NarMojo.java |4 +- nifi-parent/DISCLAIMER | 15 + nifi-parent/LICENSE | 202 +++ nifi-parent/NOTICE |5 + nifi-parent/README.md | 73 + nifi-parent/pom.xml | 431 + nifi-site/Gruntfile.js | 244 ++- nifi-site/README.md | 41 +- nifi-site/package.json | 17 +- nifi-site/src/images/flow.png | Bin 503302 - 947350 bytes nifi-site/src/includes/header.hbs |2 +- nifi-site/src/includes/topbar.hbs |5 +- .../src/pages/html/administrator-guide.hbs |7 + nifi-site/src/pages/html/developer-guide.hbs|2 +- nifi-site/src/pages/html/download.hbs | 18 + nifi-site/src/pages/html/mailing_lists.hbs |7 + nifi-site/src/pages/html/overview.hbs |2 +- nifi-site/src/pages/html/people.hbs | 27 +- nifi-site/src/pages/html/rest-api.hbs |7 + nifi-site/src/pages/html/roadmap.hbs| 30 - nifi-site/src/pages/html/screencasts.hbs| 45 +- nifi-site/src/pages/html/user-guide.hbs |2 +- nifi-site/src/pages/markdown/licensing-guide.md | 120 ++ nifi-site/src/pages/markdown/quickstart.md | 17 +- nifi-site/src/pages/markdown/release-guide.md | 126 +- nifi-site/src/scss/app.scss |2 + nifi/LICENSE| 22 + nifi/README.md | 64 +- .../annotation/behavior/DynamicProperties.java |9 +- .../annotation/behavior/DynamicProperty.java| 29 +- .../behavior/DynamicRelationship.java | 15 +- .../nifi/annotation/behavior/EventDriven.java |1 - .../annotation/behavior/ReadsAttribute.java | 19 +- .../annotation/behavior/ReadsAttributes.java| 13 +- .../annotation/behavior/SideEffectFree.java | 24 +- .../annotation/behavior/SupportsBatching.java |1 - .../annotation/behavior/TriggerSerially.java| 10 +- .../TriggerWhenAnyDestinationAvailable.java | 10 +- .../annotation/behavior/TriggerWhenEmpty.java | 15 +- .../annotation/behavior/WritesAttribute.java| 20 +- .../annotation/behavior/WritesAttributes.java | 13 +- .../documentation/CapabilityDescription.java|8 +- .../nifi/annotation/documentation/SeeAlso.java | 21 +- .../nifi/annotation/documentation/Tags.java | 14 +- .../nifi/annotation/lifecycle/OnAdded.java | 25 +- .../nifi/annotation/lifecycle/OnDisabled.java | 35 +- .../nifi/annotation/lifecycle/OnEnabled.java| 43 +- .../nifi/annotation/lifecycle/OnRemoved.java| 28 +- .../nifi/annotation/lifecycle/OnScheduled.java | 45 +- .../nifi/annotation/lifecycle/OnShutdown.java | 24 +- .../nifi/annotation/lifecycle/OnStopped.java| 25 +- .../annotation/lifecycle/OnUnscheduled.java | 31 +- .../notification/OnPrimaryNodeStateChange.java | 44 + .../notification/PrimaryNodeState.java | 33 + .../apache/nifi/authorization/Authority.java| 12 +- .../nifi/authorization/AuthorityProvider.java | 99 +- .../AuthorityProviderConfigurationContext.java | 12 +- .../authorization/DownloadAuthorization.java| 33 +- .../annotation/AuthorityProviderContext.java|1 - .../exception/ProviderCreationException.java|1 - .../exception/ProviderDestructionException.java |1 - .../AbstractConfigurableComponent.java | 14 +- .../apache/nifi/components/AllowableValue.java | 30 +- .../nifi/components/ConfigurableComponent.java | 16 +- .../nifi/components/PropertyDescriptor.java | 79 +- .../apache/nifi/components/PropertyValue.java | 40 +- .../nifi/components/ValidationContext.java | 73 +- .../nifi/components/ValidationResult.java | 17 +-
[04/10] incubator-nifi git commit: Merge pull request #3 from apache/develop
Merge pull request #3 from apache/develop Merging in latest develop branch Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/b71b5197 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b71b5197 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b71b5197 Branch: refs/heads/develop Commit: b71b51976eb6e5afba636c8dd16e33a7436e3999 Parents: 9653770 483958b Author: Brian Ghigiarelli briang...@gmail.com Authored: Thu May 14 08:57:25 2015 -0400 Committer: Brian Ghigiarelli briang...@gmail.com Committed: Thu May 14 08:57:25 2015 -0400 -- nifi-site/Gruntfile.js | 23 ++-- nifi-site/README.md| 3 +- nifi-site/src/pages/html/index.hbs | 179 3 files changed, 37 insertions(+), 168 deletions(-) --
[07/10] incubator-nifi git commit: Merge branch 'develop' into NIFI-413-PutKafka-Compression-and-Batching
Merge branch 'develop' into NIFI-413-PutKafka-Compression-and-Batching Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5b0648cf Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5b0648cf Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5b0648cf Branch: refs/heads/develop Commit: 5b0648cf3f98c91b5d392eff3cd9fd22422abeef Parents: b71b519 3045a52 Author: Brian Ghigiarelli briang...@gmail.com Authored: Fri Jun 19 17:46:53 2015 -0400 Committer: Brian Ghigiarelli briang...@gmail.com Committed: Fri Jun 19 17:46:53 2015 -0400 -- README.md | 4 +- nifi-nar-maven-plugin/pom.xml | 4 +- nifi-parent/pom.xml | 2 +- nifi-site/Gruntfile.js | 61 ++- nifi-site/src/images/flow-th.png| Bin 0 - 144319 bytes nifi-site/src/includes/topbar.hbs | 7 +- .../src/pages/html/administrator-guide.hbs | 7 - nifi-site/src/pages/html/developer-guide.hbs| 2 +- nifi-site/src/pages/html/docs.hbs | 7 + nifi-site/src/pages/html/download.hbs | 19 + nifi-site/src/pages/html/faq.hbs| 6 +- nifi-site/src/pages/html/index.hbs | 4 +- nifi-site/src/pages/html/overview.hbs | 7 - nifi-site/src/pages/html/people.hbs | 10 + nifi-site/src/pages/html/rest-api.hbs | 7 - nifi-site/src/pages/html/user-guide.hbs | 7 - nifi/nifi-api/pom.xml | 2 +- .../apache/nifi/controller/FlowFileQueue.java | 9 + nifi/nifi-assembly/LICENSE | 64 ++- nifi/nifi-assembly/NOTICE | 124 - nifi/nifi-assembly/pom.xml | 40 +- .../src/main/assembly/dependencies.xml | 2 + nifi/nifi-bootstrap/pom.xml | 9 +- .../org/apache/nifi/bootstrap/NiFiListener.java | 13 +- .../java/org/apache/nifi/bootstrap/RunNiFi.java | 423 +++--- .../org/apache/nifi/bootstrap/ShutdownHook.java | 6 +- .../nifi-data-provenance-utils/pom.xml | 2 +- .../nifi-expression-language/pom.xml| 2 +- .../attribute/expression/language/Query.java| 135 ++--- .../expression/language/TestQuery.java | 54 +- .../nifi-commons/nifi-flowfile-packager/pom.xml | 2 +- .../nifi-hl7-query-language/pom.xml | 7 +- nifi/nifi-commons/nifi-logging-utils/pom.xml| 2 +- .../nifi-processor-utilities/pom.xml| 2 +- .../nifi/processor/util/SSLProperties.java | 10 +- nifi/nifi-commons/nifi-properties/pom.xml | 2 +- .../apache/nifi/util/NiFiPropertiesTest.java| 13 +- nifi/nifi-commons/nifi-security-utils/pom.xml | 2 +- .../nifi/security/util/SslContextFactory.java | 15 +- .../nifi-site-to-site-client/pom.xml| 4 +- nifi/nifi-commons/nifi-socket-utils/pom.xml | 2 +- .../apache/nifi/io/nio/ChannelDispatcher.java | 6 +- .../org/apache/nifi/io/nio/ChannelListener.java | 4 +- .../nifi/io/nio/DatagramChannelReader.java | 8 +- .../apache/nifi/io/nio/example/ServerMain.java | 2 +- nifi/nifi-commons/nifi-utils/pom.xml| 4 +- nifi/nifi-commons/nifi-web-utils/pom.xml| 2 +- nifi/nifi-commons/nifi-write-ahead-log/pom.xml | 2 +- nifi/nifi-commons/pom.xml | 2 +- nifi/nifi-docs/pom.xml | 4 +- nifi/nifi-external/nifi-spark-receiver/pom.xml | 3 +- nifi/nifi-external/nifi-storm-spout/pom.xml | 38 ++ .../org/apache/nifi/storm/NiFiDataPacket.java | 39 ++ .../java/org/apache/nifi/storm/NiFiSpout.java | 232 nifi/nifi-external/pom.xml | 3 +- .../nifi-processor-bundle-archetype/pom.xml | 2 +- nifi/nifi-maven-archetypes/pom.xml | 2 +- nifi/nifi-mock/pom.xml | 2 +- .../apache/nifi/util/MockProcessSession.java| 40 +- .../nifi/util/MockProvenanceReporter.java | 365 +++-- .../apache/nifi/util/MockSessionFactory.java| 8 +- .../apache/nifi/util/SharedSessionState.java| 26 +- .../nifi/util/StandardProcessorTestRunner.java | 34 +- .../java/org/apache/nifi/util/TestRunner.java | 15 + .../nifi-aws-bundle/nifi-aws-nar/pom.xml| 72 +-- .../nifi-aws-bundle/nifi-aws-processors/pom.xml | 142 ++--- nifi/nifi-nar-bundles/nifi-aws-bundle/pom.xml | 86 +-- .../nifi-framework-nar/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 27 +- .../nifi-framework/nifi-administration/pom.xml | 2 +- .../nifi-framework/nifi-client-dto/pom.xml | 2 +- .../nifi-cluster-authorization-provider/pom.xml | 2 +- .../nifi-framework/nifi-documentation/pom.xml | 80
[10/10] incubator-nifi git commit: NIFI-413: Formatted code to fix checkstyle failures
NIFI-413: Formatted code to fix checkstyle failures Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f2f90560 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f2f90560 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f2f90560 Branch: refs/heads/develop Commit: f2f90560557de70ef4404672ce53c4593995a5f1 Parents: f5226ad Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 22 12:05:02 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 12:05:02 2015 -0400 -- .../apache/nifi/processors/kafka/PutKafka.java | 319 ++- .../nifi/processors/kafka/TestPutKafka.java | 79 ++--- 2 files changed, 203 insertions(+), 195 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f2f90560/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java -- diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index e572622..d83c7bf 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -61,7 +61,7 @@ import org.apache.nifi.util.LongHolder; import scala.actors.threadpool.Arrays; @SupportsBatching -@Tags({Apache, Kafka, Put, Send, Message, PubSub}) +@Tags({ Apache, Kafka, Put, Send, Message, PubSub }) @CapabilityDescription(Sends the contents of a FlowFile as a message to Apache Kafka) public class PutKafka extends AbstractProcessor { @@ -69,13 +69,13 @@ public class PutKafka extends AbstractProcessor { private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + (?:,\\s* + SINGLE_BROKER_REGEX + )*; public static final AllowableValue DELIVERY_REPLICATED = new AllowableValue(-1, Guarantee Replicated Delivery, FlowFile will be routed to -+ failure unless the message is replicated to the appropriate number of Kafka Nodes according to the Topic configuration); ++ failure unless the message is replicated to the appropriate number of Kafka Nodes according to the Topic configuration); public static final AllowableValue DELIVERY_ONE_NODE = new AllowableValue(1, Guarantee Single Node Delivery, FlowFile will be routed -+ to success if the message is received by a single Kafka node, whether or not it is replicated. This is faster than -+ Guarantee Replicated Delivery but can result in data loss if a Kafka node crashes); ++ to success if the message is received by a single Kafka node, whether or not it is replicated. This is faster than ++ Guarantee Replicated Delivery but can result in data loss if a Kafka node crashes); public static final AllowableValue DELIVERY_BEST_EFFORT = new AllowableValue(0, Best Effort, FlowFile will be routed to success after -+ successfully writing the content to a Kafka node, without waiting for a response. This provides the best performance but may result -+ in data loss.); ++ successfully writing the content to a Kafka node, without waiting for a response. This provides the best performance but may result ++ in data loss.); /** * AllowableValue for a Producer Type that synchronously sends messages to Kafka @@ -86,7 +86,7 @@ public class PutKafka extends AbstractProcessor { * AllowableValue for a Producer Type that asynchronously sends messages to Kafka */ public static final AllowableValue PRODUCTER_TYPE_ASYNCHRONOUS = new AllowableValue(async, Asynchronous, Batch messages before sending them to Kafka. - + While this will improve throughput, it opens the possibility that a failure on the client machine will drop unsent data.); ++ While this will improve throughput, it opens the possibility that a failure on the client machine will drop unsent data.); /** * AllowableValue for sending messages to Kafka without compression @@ -103,150 +103,156 @@ public class PutKafka extends AbstractProcessor { */ public static final AllowableValue COMPRESSION_CODEC_SNAPPY = new AllowableValue(snappy, Snappy, Compress messages using Snappy); + public static final PropertyDescriptor SEED_BROKERS = new PropertyDescriptor.Builder() -.name(Known Brokers) -.description(A comma-separated list of
[03/10] incubator-nifi git commit: [NIFI-413] Adding properties to PutKafka to support asynchronous production with configurable batching. Also added user-defined control over compression codec and co
[NIFI-413] Adding properties to PutKafka to support asynchronous production with configurable batching. Also added user-defined control over compression codec and compressed topics. Producer type remains synchronous by default. Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/9653770a Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/9653770a Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/9653770a Branch: refs/heads/develop Commit: 9653770ac4a050f4439c07f7ae6e34658f08e5a0 Parents: 421ad8f Author: Brian Ghigiarelli briang...@gmail.com Authored: Thu May 14 08:55:04 2015 -0400 Committer: Brian Ghigiarelli briang...@gmail.com Committed: Thu May 14 08:55:04 2015 -0400 -- .../apache/nifi/processors/kafka/PutKafka.java | 81 - .../nifi/processors/kafka/TestPutKafka.java | 93 +++- 2 files changed, 171 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9653770a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java -- diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index 44b6584..5bd0d2b 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -136,6 +136,68 @@ public class PutKafka extends AbstractProcessor { .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(false) .build(); +public static final PropertyDescriptor PRODUCER_TYPE = new PropertyDescriptor.Builder() +.name(Producer Type) +.description(This parameter specifies whether the messages are sent asynchronously in a background thread. ++ Valid values are (1) async for asynchronous send and (2) sync for synchronous send. ++ By setting the producer to async we allow batching together of requests (which is great for throughput) ++ but open the possibility of a failure of the client machine dropping unsent data.) +.required(true) +.allowableValues(sync, async) +.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.expressionLanguageSupported(false) +.defaultValue(sync) +.build(); +public static final PropertyDescriptor BATCH_NUM_MESSAGES = new PropertyDescriptor.Builder() +.name(Async Message Batch Size (batch.num.messages)) +.description(Used only if Producer Type is set to \async\. The number of messages to send in one batch when using async mode. ++ The producer will wait until either this number of messages are ready ++ to send or queue.buffer.max.ms is reached.) +.required(true) +.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) +.defaultValue(200).build(); +public static final PropertyDescriptor QUEUE_BUFFERING_MAX_MS = new PropertyDescriptor.Builder() +.name(Queue Buffering Max Time (queue.buffering.max.ms)) +.description(Used only if Producer Type is set to \async\. Maximum time to buffer data when using async mode. For example a setting of 100 ++ will try to batch together 100ms of messages to send at once. This will improve ++ throughput but adds message delivery latency due to the buffering.) +.required(true) +.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) +.defaultValue(5000).build(); +public static final PropertyDescriptor QUEUE_BUFFERING_MAX_MESSAGES = new PropertyDescriptor.Builder() +.name(Queue Buffer Max Count (queue.buffering.max.messages)) +.description(Used only if Producer Type is set to \async\. The maximum number of unsent messages that can be queued up the producer when ++ using async mode before either the producer must be blocked or data must be dropped.) +.required(true) +.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) +.defaultValue(1).build(); +public static final PropertyDescriptor QUEUE_ENQUEUE_TIMEOUT_MS = new
[08/10] incubator-nifi git commit: [NIFI-413] Updating PutKafka properties to follow NiFi standards. Added validation for asynchronous properties.
[NIFI-413] Updating PutKafka properties to follow NiFi standards. Added validation for asynchronous properties. Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/8af84f3f Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/8af84f3f Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/8af84f3f Branch: refs/heads/develop Commit: 8af84f3f73bd7890c44c1f3b7597330023e34d16 Parents: 5b0648c Author: Brian Ghigiarelli briang...@gmail.com Authored: Fri Jun 19 18:49:21 2015 -0400 Committer: Brian Ghigiarelli briang...@gmail.com Committed: Fri Jun 19 18:49:21 2015 -0400 -- .../apache/nifi/processors/kafka/PutKafka.java | 129 +-- .../nifi/processors/kafka/TestPutKafka.java | 78 ++- 2 files changed, 167 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8af84f3f/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java -- diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index 5bd0d2b..e572622 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Properties; @@ -39,6 +40,8 @@ import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnStopped; 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.flowfile.FlowFile; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.DataUnit; @@ -74,6 +77,32 @@ public class PutKafka extends AbstractProcessor { + successfully writing the content to a Kafka node, without waiting for a response. This provides the best performance but may result + in data loss.); +/** + * AllowableValue for a Producer Type that synchronously sends messages to Kafka + */ +public static final AllowableValue PRODUCTER_TYPE_SYNCHRONOUS = new AllowableValue(sync, Synchronous, Send FlowFiles to Kafka immediately.); + +/** + * AllowableValue for a Producer Type that asynchronously sends messages to Kafka + */ +public static final AllowableValue PRODUCTER_TYPE_ASYNCHRONOUS = new AllowableValue(async, Asynchronous, Batch messages before sending them to Kafka. + + While this will improve throughput, it opens the possibility that a failure on the client machine will drop unsent data.); + +/** + * AllowableValue for sending messages to Kafka without compression + */ +public static final AllowableValue COMPRESSION_CODEC_NONE = new AllowableValue(none, None, Compression will not be used for any topic.); + +/** + * AllowableValue for sending messages to Kafka with GZIP compression + */ +public static final AllowableValue COMPRESSION_CODEC_GZIP = new AllowableValue(gzip, GZIP, Compress messages using GZIP); + +/** + * AllowableValue for sending messages to Kafka with Snappy compression + */ +public static final AllowableValue COMPRESSION_CODEC_SNAPPY = new AllowableValue(snappy, Snappy, Compress messages using Snappy); + public static final PropertyDescriptor SEED_BROKERS = new PropertyDescriptor.Builder() .name(Known Brokers) .description(A comma-separated list of known Kafka Brokers in the format host:port) @@ -138,64 +167,66 @@ public class PutKafka extends AbstractProcessor { .build(); public static final PropertyDescriptor PRODUCER_TYPE = new PropertyDescriptor.Builder() .name(Producer Type) -.description(This parameter specifies whether the messages are sent asynchronously in a background thread. -+ Valid values are (1) async for asynchronous send and (2) sync for synchronous send. -+ By setting the producer to async we allow batching together of requests (which is great for throughput) -
[09/10] incubator-nifi git commit: Merge branch 'NIFI-413-PutKafka-Compression-and-Batching' of https://github.com/brianghig/incubator-nifi into NIFI-413
Merge branch 'NIFI-413-PutKafka-Compression-and-Batching' of https://github.com/brianghig/incubator-nifi into NIFI-413 Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f5226ad3 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f5226ad3 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f5226ad3 Branch: refs/heads/develop Commit: f5226ad3c6da00197b62cabd935e0e2d0c453cd9 Parents: c4c98fc 8af84f3 Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 22 10:37:42 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 10:37:42 2015 -0400 -- .../apache/nifi/processors/kafka/PutKafka.java | 136 ++- .../nifi/processors/kafka/TestPutKafka.java | 164 ++- 2 files changed, 298 insertions(+), 2 deletions(-) --
[06/10] incubator-nifi git commit: Merge branch 'develop' of github.com:apache/incubator-nifi into develop
Merge branch 'develop' of github.com:apache/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/3045a52e Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/3045a52e Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/3045a52e Branch: refs/heads/develop Commit: 3045a52e354dd1f4cec380747b3d9cd6b1feba56 Parents: ce16aab 531136d Author: Brian Ghigiarelli briang...@gmail.com Authored: Fri Jun 19 17:46:08 2015 -0400 Committer: Brian Ghigiarelli briang...@gmail.com Committed: Fri Jun 19 17:46:08 2015 -0400 -- nifi-site/Gruntfile.js | 51 +- nifi-site/src/includes/topbar.hbs | 7 +- .../src/pages/html/administrator-guide.hbs | 7 - nifi-site/src/pages/html/developer-guide.hbs| 2 +- nifi-site/src/pages/html/docs.hbs | 7 + nifi-site/src/pages/html/faq.hbs| 6 +- nifi-site/src/pages/html/overview.hbs | 7 - nifi-site/src/pages/html/people.hbs | 10 + nifi-site/src/pages/html/rest-api.hbs | 7 - nifi-site/src/pages/html/user-guide.hbs | 7 - nifi/nifi-api/pom.xml | 2 +- .../apache/nifi/controller/FlowFileQueue.java | 9 + nifi/nifi-assembly/LICENSE | 37 +- nifi/nifi-assembly/NOTICE | 120 +++- nifi/nifi-assembly/pom.xml | 28 +- .../src/main/assembly/dependencies.xml | 2 + nifi/nifi-bootstrap/pom.xml | 9 +- .../org/apache/nifi/bootstrap/NiFiListener.java | 13 +- .../java/org/apache/nifi/bootstrap/RunNiFi.java | 423 +++--- .../org/apache/nifi/bootstrap/ShutdownHook.java | 6 +- .../nifi-data-provenance-utils/pom.xml | 2 +- .../nifi-expression-language/pom.xml| 2 +- .../attribute/expression/language/Query.java| 135 ++--- .../expression/language/TestQuery.java | 54 +- .../nifi-commons/nifi-flowfile-packager/pom.xml | 2 +- .../nifi-hl7-query-language/pom.xml | 4 +- nifi/nifi-commons/nifi-logging-utils/pom.xml| 2 +- .../nifi-processor-utilities/pom.xml| 2 +- .../nifi/processor/util/SSLProperties.java | 10 +- nifi/nifi-commons/nifi-properties/pom.xml | 2 +- .../apache/nifi/util/NiFiPropertiesTest.java| 13 +- nifi/nifi-commons/nifi-security-utils/pom.xml | 2 +- .../nifi/security/util/SslContextFactory.java | 15 +- .../nifi-site-to-site-client/pom.xml| 4 +- nifi/nifi-commons/nifi-socket-utils/pom.xml | 2 +- nifi/nifi-commons/nifi-utils/pom.xml| 4 +- nifi/nifi-commons/nifi-web-utils/pom.xml| 2 +- nifi/nifi-commons/nifi-write-ahead-log/pom.xml | 2 +- nifi/nifi-commons/pom.xml | 2 +- nifi/nifi-docs/pom.xml | 4 +- nifi/nifi-external/nifi-spark-receiver/pom.xml | 3 +- nifi/nifi-external/nifi-storm-spout/pom.xml | 38 ++ .../org/apache/nifi/storm/NiFiDataPacket.java | 39 ++ .../java/org/apache/nifi/storm/NiFiSpout.java | 232 nifi/nifi-external/pom.xml | 3 +- .../nifi-processor-bundle-archetype/pom.xml | 2 +- nifi/nifi-maven-archetypes/pom.xml | 2 +- nifi/nifi-mock/pom.xml | 2 +- .../apache/nifi/util/MockProcessSession.java| 40 +- .../nifi/util/MockProvenanceReporter.java | 365 +++-- .../apache/nifi/util/MockSessionFactory.java| 8 +- .../apache/nifi/util/SharedSessionState.java| 26 +- .../nifi/util/StandardProcessorTestRunner.java | 34 +- .../java/org/apache/nifi/util/TestRunner.java | 15 + .../nifi-aws-bundle/nifi-aws-nar/pom.xml| 4 +- .../nifi-aws-bundle/nifi-aws-processors/pom.xml | 4 +- nifi/nifi-nar-bundles/nifi-aws-bundle/pom.xml | 2 +- .../nifi-framework-nar/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 27 +- .../nifi-framework/nifi-administration/pom.xml | 2 +- .../nifi-framework/nifi-client-dto/pom.xml | 2 +- .../nifi-cluster-authorization-provider/pom.xml | 2 +- .../nifi-framework/nifi-documentation/pom.xml | 2 +- ...kControllerServiceInitializationContext.java | 4 +- .../MockProcessorInitializationContext.java | 4 +- .../documentation/mock/MockProcessorLogger.java | 169 ++ .../MockReportingInitializationContext.java | 4 +- .../example/ControllerServiceWithLogger.java| 31 ++ .../example/ProcessorWithLogger.java| 37 ++ .../example/ReportingTaskWithLogger.java| 36 ++ .../html/HtmlDocumentationWriterTest.java | 40 +- .../html/ProcessorDocumentationWriterTest.java | 18 +- .../nifi/documentation/html/XmlValidator.java | 4 +-
incubator-nifi git commit: NIFI-545: Code cleanup
Repository: incubator-nifi Updated Branches: refs/heads/develop b22a1261c - 979671ca9 NIFI-545: Code cleanup Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/979671ca Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/979671ca Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/979671ca Branch: refs/heads/develop Commit: 979671ca9719aca6789f213529991fc99d86979d Parents: b22a126 Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 22 09:21:44 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 15:58:08 2015 -0400 -- .../nifi/cluster/flow/impl/DataFlowDaoImpl.java | 15 +++ 1 file changed, 3 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/979671ca/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java index c0395a4..335c0ef 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java @@ -188,7 +188,7 @@ public class DataFlowDaoImpl implements DataFlowDao { return; } -if ((primaryEntry == null restoreEntry != null) || (primaryEntry != null restoreEntry == null)) { +if (primaryEntry == null restoreEntry != null || primaryEntry != null restoreEntry == null) { throw new IllegalStateException(String.format(Primary file '%s' is different than restore file '%s', primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath())); } @@ -352,7 +352,7 @@ public class DataFlowDaoImpl implements DataFlowDao { final File[] files = dir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { -return (name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT)); +return name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT); } }); @@ -515,19 +515,10 @@ public class DataFlowDaoImpl implements DataFlowDao { final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); dataFlow.setAutoStartProcessors(autoStart); -return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes); +return new ClusterDataFlow(dataFlow, clusterMetadata == null ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes); } private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException { - -// get the data flow -DataFlow dataFlow = clusterDataFlow.getDataFlow(); - -// if no dataflow, then write a new dataflow -if (dataFlow == null) { -dataFlow = new StandardDataFlow(new byte[0], new byte[0], new byte[0]); -} - // setup the cluster metadata final ClusterMetadata clusterMetadata = new ClusterMetadata(); clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId());
incubator-nifi git commit: NIFI-80: Truncate attribute values that exceed some threshold. Expose threshold as properties in nifi.properties file
Repository: incubator-nifi Updated Branches: refs/heads/develop 979671ca9 - 25146a582 NIFI-80: Truncate attribute values that exceed some threshold. Expose threshold as properties in nifi.properties file Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/25146a58 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/25146a58 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/25146a58 Branch: refs/heads/develop Commit: 25146a5828bc5dd1d04c5252843bef93ea87afac Parents: 979671c Author: Mark Payne marka...@hotmail.com Authored: Sat Jun 20 08:52:14 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 16:06:40 2015 -0400 -- nifi/nifi-assembly/pom.xml | 1 + .../src/main/resources/conf/nifi.properties | 3 + .../nifi/provenance/IndexConfiguration.java | 2 +- .../PersistentProvenanceRepository.java | 90 +--- .../provenance/RepositoryConfiguration.java | 18 .../nifi/provenance/StandardRecordReader.java | 15 ++-- .../provenance/lucene/DeleteIndexAction.java| 6 +- .../nifi/provenance/lucene/DocsReader.java | 9 +- .../nifi/provenance/lucene/IndexSearch.java | 7 +- .../nifi/provenance/lucene/LineageQuery.java| 6 +- .../provenance/serialization/RecordReaders.java | 17 +++- .../TestPersistentProvenanceRepository.java | 39 - .../TestStandardRecordReaderWriter.java | 10 +-- 13 files changed, 180 insertions(+), 43 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/25146a58/nifi/nifi-assembly/pom.xml -- diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index 02a16f9..9b17617 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -276,6 +276,7 @@ language governing permissions and limitations under the License. -- nifi.provenance.repository.index.shard.size500 MB/nifi.provenance.repository.index.shard.size nifi.provenance.repository.always.syncfalse/nifi.provenance.repository.always.sync nifi.provenance.repository.journal.count16/nifi.provenance.repository.journal.count + nifi.provenance.repository.max.attribute.length65536/nifi.provenance.repository.max.attribute.length !-- volatile provenance repository properties -- nifi.provenance.repository.buffer.size10/nifi.provenance.repository.buffer.size http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/25146a58/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties index 90b3cdd..4043076 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties @@ -82,6 +82,9 @@ nifi.provenance.repository.indexed.attributes=${nifi.provenance.repository.index # Large values for the shard size will result in more Java heap usage when searching the Provenance Repository # but should provide better performance nifi.provenance.repository.index.shard.size=${nifi.provenance.repository.index.shard.size} +# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from +# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved. +nifi.provenance.repository.max.attribute.length=${nifi.provenance.repository.max.attribute.length} # Volatile Provenance Respository Properties nifi.provenance.repository.buffer.size=${nifi.provenance.repository.buffer.size} http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/25146a58/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java -- diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java index 9ea793d..4e80811 100644 ---
[2/2] incubator-nifi git commit: NIFI-482: Allowed for new 'literal' function. Make expressions invalid when they attempt to evaluate functions against the values of 'reducing functions'; cleanup Eval
NIFI-482: Allowed for new 'literal' function. Make expressions invalid when they attempt to evaluate functions against the values of 'reducing functions'; cleanup Evaluators that were depending on Abstract classes instead of interfaces. Updated EL Guide to include new 'literal' function Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/c4c98fc4 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c4c98fc4 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c4c98fc4 Branch: refs/heads/develop Commit: c4c98fc4a92ef44993324c5510cc31ba4736d4b8 Parents: 0f01b3c Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 19 09:01:35 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 22 09:04:55 2015 -0400 -- .../language/antlr/AttributeExpressionLexer.g | 1 + .../language/antlr/AttributeExpressionParser.g | 3 +- .../attribute/expression/language/Query.java| 255 --- .../language/evaluation/BooleanEvaluator.java | 11 + .../language/evaluation/DateEvaluator.java | 11 + .../language/evaluation/Evaluator.java | 12 + .../language/evaluation/NumberEvaluator.java| 10 + .../language/evaluation/StringEvaluator.java| 11 + .../evaluation/functions/AndEvaluator.java | 6 +- .../evaluation/functions/AppendEvaluator.java | 6 +- .../functions/AttributeEvaluator.java | 4 +- .../evaluation/functions/ContainsEvaluator.java | 7 +- .../evaluation/functions/DivideEvaluator.java | 6 +- .../evaluation/functions/EndsWithEvaluator.java | 7 +- .../evaluation/functions/FindEvaluator.java | 7 +- .../evaluation/functions/FormatEvaluator.java | 4 +- .../functions/GreaterThanEvaluator.java | 7 +- .../functions/GreaterThanOrEqualEvaluator.java | 7 +- .../evaluation/functions/IndexOfEvaluator.java | 7 +- .../functions/LastIndexOfEvaluator.java | 7 +- .../evaluation/functions/LengthEvaluator.java | 7 +- .../evaluation/functions/LessThanEvaluator.java | 7 +- .../functions/LessThanOrEqualEvaluator.java | 7 +- .../evaluation/functions/MatchesEvaluator.java | 7 +- .../evaluation/functions/MinusEvaluator.java| 6 +- .../evaluation/functions/ModEvaluator.java | 6 +- .../evaluation/functions/MultiplyEvaluator.java | 6 +- .../evaluation/functions/NotEvaluator.java | 4 +- .../functions/NumberToDateEvaluator.java| 5 +- .../evaluation/functions/OrEvaluator.java | 6 +- .../evaluation/functions/PlusEvaluator.java | 6 +- .../evaluation/functions/PrependEvaluator.java | 6 +- .../functions/ReplaceAllEvaluator.java | 8 +- .../functions/ReplaceEmptyEvaluator.java| 6 +- .../evaluation/functions/ReplaceEvaluator.java | 8 +- .../functions/ReplaceNullEvaluator.java | 6 +- .../functions/StartsWithEvaluator.java | 7 +- .../functions/StringToDateEvaluator.java| 7 +- .../functions/SubstringAfterEvaluator.java | 6 +- .../functions/SubstringAfterLastEvaluator.java | 6 +- .../functions/SubstringBeforeEvaluator.java | 6 +- .../functions/SubstringBeforeLastEvaluator.java | 6 +- .../functions/SubstringEvaluator.java | 11 +- .../evaluation/functions/ToLowerEvaluator.java | 6 +- .../evaluation/functions/ToNumberEvaluator.java | 7 +- .../evaluation/functions/ToRadixEvaluator.java | 11 +- .../evaluation/functions/ToUpperEvaluator.java | 6 +- .../evaluation/functions/TrimEvaluator.java | 4 +- .../functions/UrlDecodeEvaluator.java | 6 +- .../functions/UrlEncodeEvaluator.java | 6 +- .../evaluation/literals/ToLiteralEvaluator.java | 43 .../evaluation/reduce/JoinEvaluator.java| 6 +- .../selection/AllAttributesEvaluator.java | 7 +- .../selection/AnyAttributeEvaluator.java| 7 +- .../AnyMatchingAttributeEvaluator.java | 21 -- .../selection/DelineatedAttributeEvaluator.java | 12 +- .../selection/IteratingEvaluator.java | 33 +++ .../evaluation/selection/MappingEvaluator.java | 11 +- .../selection/MultiAttributeEvaluator.java | 3 +- .../selection/MultiMatchAttributeEvaluator.java | 5 + .../selection/MultiNamedAttributeEvaluator.java | 5 + .../expression/language/TestQuery.java | 54 +++- .../asciidoc/expression-language-guide.adoc | 26 +- 63 files changed, 532 insertions(+), 286 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c4c98fc4/nifi/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g -- diff
[1/2] incubator-nifi git commit: NIFI-482: Allowed for new 'literal' function. Make expressions invalid when they attempt to evaluate functions against the values of 'reducing functions'; cleanup Eval
Repository: incubator-nifi Updated Branches: refs/heads/develop 0f01b3c96 - c4c98fc4a http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c4c98fc4/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java -- diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java index dff693d..7c9278f 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; public class PlusEvaluator extends NumberEvaluator { -private final NumberEvaluator subject; -private final NumberEvaluator plusValue; +private final EvaluatorLong subject; +private final EvaluatorLong plusValue; -public PlusEvaluator(final NumberEvaluator subject, final NumberEvaluator plusValue) { +public PlusEvaluator(final EvaluatorLong subject, final EvaluatorLong plusValue) { this.subject = subject; this.plusValue = plusValue; } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c4c98fc4/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java -- diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java index 5b66b8f..1053328 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java @@ -25,10 +25,10 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class PrependEvaluator extends StringEvaluator { -private final StringEvaluator subject; -private final StringEvaluator prependEvaluator; +private final EvaluatorString subject; +private final EvaluatorString prependEvaluator; -public PrependEvaluator(final StringEvaluator subject, final StringEvaluator prepend) { +public PrependEvaluator(final EvaluatorString subject, final EvaluatorString prepend) { this.subject = subject; this.prependEvaluator = prepend; } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c4c98fc4/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java -- diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java index 19ed63e..b796233 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java @@ -25,11 +25,11 @@ import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResul public class ReplaceAllEvaluator extends StringEvaluator { -private final StringEvaluator subject; -private final StringEvaluator search; -private final StringEvaluator replacement; +private final EvaluatorString subject; +private final EvaluatorString search; +private final EvaluatorString replacement; -public ReplaceAllEvaluator(final StringEvaluator subject, final StringEvaluator search, final StringEvaluator replacement) { +public ReplaceAllEvaluator(final EvaluatorString subject, final EvaluatorString search, final EvaluatorString replacement) { this.subject = subject; this.search = search; this.replacement = replacement;
incubator-nifi git commit: NIFI-707: Use HTTP Content-Type Header to set mime.type attribute for GetHTTP
Repository: incubator-nifi Updated Branches: refs/heads/develop 33848b303 - 0f01b3c96 NIFI-707: Use HTTP Content-Type Header to set mime.type attribute for GetHTTP Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/0f01b3c9 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/0f01b3c9 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/0f01b3c9 Branch: refs/heads/develop Commit: 0f01b3c96fb0a12df7fe5159adf99d899eef46d5 Parents: 33848b3 Author: Mark Payne marka...@hotmail.com Authored: Sat Jun 20 17:20:42 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sat Jun 20 17:20:42 2015 -0400 -- .../apache/nifi/processors/standard/GetHTTP.java | 17 +++-- 1 file changed, 15 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0f01b3c9/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index 1654a4f..0001bb3 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -67,9 +67,10 @@ import org.apache.http.conn.ssl.TrustSelfSignedStrategy; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.conn.BasicHttpClientConnectionManager; +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.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; @@ -91,7 +92,10 @@ import org.apache.nifi.util.StopWatch; @Tags({get, fetch, poll, http, https, ingest, source, input}) @CapabilityDescription(Fetches a file via HTTP) -@WritesAttribute(attribute = filename, description = the filename is set to the name of the file on the remote server) +@WritesAttributes({ +@WritesAttribute(attribute = filename, description = The filename is set to the name of the file on the remote server), +@WritesAttribute(attribute = mime.type, description = The MIME Type of the FlowFile, as reported by the HTTP Content-Type header) +}) public class GetHTTP extends AbstractSessionFactoryProcessor { static final int PERSISTENCE_INTERVAL_MSEC = 1; @@ -413,6 +417,15 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), context.getProperty(FILENAME).getValue()); flowFile = session.putAttribute(flowFile, this.getClass().getSimpleName().toLowerCase() + .remote.source, source); flowFile = session.importFrom(response.getEntity().getContent(), flowFile); + +final Header contentTypeHeader = response.getFirstHeader(Content-Type); +if (contentTypeHeader != null) { +final String contentType = contentTypeHeader.getValue(); +if (!contentType.trim().isEmpty()) { +flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), contentType.trim()); +} +} + final long flowFileSize = flowFile.getSize(); stopWatch.stop(); final String dataRate = stopWatch.calculateDataRate(flowFileSize);
incubator-nifi git commit: NIFI-702: Ensure that we always skip to the appropriate location within the ContentClaim when exporting to an external stream or Path
Repository: incubator-nifi Updated Branches: refs/heads/develop cef720679 - a1f043845 NIFI-702: Ensure that we always skip to the appropriate location within the ContentClaim when exporting to an external stream or Path Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a1f04384 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a1f04384 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a1f04384 Branch: refs/heads/develop Commit: a1f0438451e5893770616e2829a3f4ea034f625f Parents: cef7206 Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 19 11:10:11 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jun 19 11:10:11 2015 -0400 -- .../repository/VolatileContentRepository.java | 30 1 file changed, 12 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a1f04384/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java index 5971865..e4fa428 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java @@ -23,7 +23,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.StandardCopyOption; import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Collection; @@ -44,11 +43,10 @@ import org.apache.nifi.controller.repository.claim.StandardContentClaim; import org.apache.nifi.controller.repository.io.ArrayManagedOutputStream; import org.apache.nifi.controller.repository.io.MemoryManager; import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.processor.DataUnit; import org.apache.nifi.stream.io.ByteArrayInputStream; import org.apache.nifi.stream.io.StreamUtils; -import org.apache.nifi.processor.DataUnit; import org.apache.nifi.util.NiFiProperties; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -372,20 +370,16 @@ public class VolatileContentRepository implements ContentRepository { return 0L; } -if (append) { -try (final InputStream in = read(claim); -final OutputStream destinationStream = Files.newOutputStream(destination, StandardOpenOption.APPEND)) { +final StandardOpenOption openOption = append ? StandardOpenOption.APPEND : StandardOpenOption.CREATE; +try (final InputStream in = read(claim); +final OutputStream destinationStream = Files.newOutputStream(destination, openOption)) { -if (offset 0) { -StreamUtils.skip(in, offset); -} - -StreamUtils.copy(in, destinationStream, length); -return length; +if (offset 0) { +StreamUtils.skip(in, offset); } -} else { -Files.copy(read(claim), destination, StandardCopyOption.REPLACE_EXISTING); -return Files.size(destination); + +StreamUtils.copy(in, destinationStream, length); +return length; } } @@ -419,7 +413,7 @@ public class VolatileContentRepository implements ContentRepository { } final ContentClaim backupClaim = getBackupClaim(claim); -return (backupClaim == null) ? getContent(claim).getSize() : getBackupRepository().size(claim); +return backupClaim == null ? getContent(claim).getSize() : getBackupRepository().size(claim); } @Override @@ -429,13 +423,13 @@ public class VolatileContentRepository implements ContentRepository { } final ContentClaim backupClaim = getBackupClaim(claim); -return (backupClaim == null) ? getContent(claim).read() : getBackupRepository().read(backupClaim); +return backupClaim == null ? getContent(claim).read() : getBackupRepository().read(backupClaim); } @Override public OutputStream write(final ContentClaim claim) throws IOException { final ContentClaim
[1/2] incubator-nifi git commit: NIFI-230: Fixed bug in calculating how much data has been written to a ContentClaim
Repository: incubator-nifi Updated Branches: refs/heads/develop 548f6d083 - 531136db9 NIFI-230: Fixed bug in calculating how much data has been written to a ContentClaim Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/c4d186bd Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c4d186bd Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c4d186bd Branch: refs/heads/develop Commit: c4d186bdd997f3f697bf6c9d3ed8a09bbe970e8b Parents: b7b42c7 Author: Mark Payne marka...@hotmail.com Authored: Thu Jun 18 11:30:04 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 18 11:30:04 2015 -0400 -- .../repository/VolatileContentRepository.java | 20 +++- 1 file changed, 15 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c4d186bd/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java index 5971865..e35a63c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java @@ -419,7 +419,7 @@ public class VolatileContentRepository implements ContentRepository { } final ContentClaim backupClaim = getBackupClaim(claim); -return (backupClaim == null) ? getContent(claim).getSize() : getBackupRepository().size(claim); +return backupClaim == null ? getContent(claim).getSize() : getBackupRepository().size(claim); } @Override @@ -429,13 +429,13 @@ public class VolatileContentRepository implements ContentRepository { } final ContentClaim backupClaim = getBackupClaim(claim); -return (backupClaim == null) ? getContent(claim).read() : getBackupRepository().read(backupClaim); +return backupClaim == null ? getContent(claim).read() : getBackupRepository().read(backupClaim); } @Override public OutputStream write(final ContentClaim claim) throws IOException { final ContentClaim backupClaim = getBackupClaim(claim); -return (backupClaim == null) ? getContent(claim).write() : getBackupRepository().write(backupClaim); +return backupClaim == null ? getContent(claim).write() : getBackupRepository().write(backupClaim); } @Override @@ -481,8 +481,13 @@ public class VolatileContentRepository implements ContentRepository { @Override public void write(int b) throws IOException { try { +final long bufferLengthBefore = getBufferLength(); super.write(b); -repoSizeCounter.incrementAndGet(); +final long bufferLengthAfter = getBufferLength(); +final long bufferSpaceAdded = bufferLengthAfter - bufferLengthBefore; +if (bufferSpaceAdded 0) { +repoSizeCounter.addAndGet(bufferSpaceAdded); +} } catch (final IOException e) { final byte[] buff = new byte[1]; buff[0] = (byte) (b 0xFF); @@ -498,8 +503,13 @@ public class VolatileContentRepository implements ContentRepository { @Override public void write(byte[] b, int off, int len) throws IOException { try { +final long bufferLengthBefore = getBufferLength(); super.write(b, off, len); -repoSizeCounter.addAndGet(len); +final long bufferLengthAfter = getBufferLength(); +final long bufferSpaceAdded = bufferLengthAfter - bufferLengthBefore; +if (bufferSpaceAdded 0) { +repoSizeCounter.addAndGet(bufferSpaceAdded); +} } catch (final IOException e) { redirect(b, off, len); }
[2/2] incubator-nifi git commit: Merge branch 'NIFI-230' into develop
Merge branch 'NIFI-230' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/531136db Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/531136db Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/531136db Branch: refs/heads/develop Commit: 531136db98743664fbe01f10947a7d7c26ecb4aa Parents: 548f6d0 c4d186b Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 19 13:12:09 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jun 19 13:12:09 2015 -0400 -- .../repository/VolatileContentRepository.java | 14 -- 1 file changed, 12 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/531136db/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java --
[1/2] incubator-nifi git commit: NIFI-703: Do not use prefetch if using FlowFileFilter, as this can result in FlowFiles not being pulled from the queue
Repository: incubator-nifi Updated Branches: refs/heads/develop 2d0bb1c1c - 548f6d083 NIFI-703: Do not use prefetch if using FlowFileFilter, as this can result in FlowFiles not being pulled from the queue Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a5d6f88c Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a5d6f88c Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a5d6f88c Branch: refs/heads/develop Commit: a5d6f88c2e0c6fc753fa753f259aaa6e3475049a Parents: a1f0438 Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 19 12:55:04 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jun 19 12:55:04 2015 -0400 -- .../nifi/controller/StandardFlowFileQueue.java| 18 +++--- 1 file changed, 11 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5d6f88c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java index 8f6c8ed..f47ea2f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java @@ -363,7 +363,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue { return true; } -if (maxBytes 0 (queueSize.getByteCount() = maxBytes)) { +if (maxBytes 0 queueSize.getByteCount() = maxBytes) { return true; } @@ -437,7 +437,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue { final ListFlowFileRecord swapRecords = new ArrayList(Math.min(SWAP_RECORD_POLL_SIZE, swapQueue.size())); final IteratorFlowFileRecord itr = swapQueue.iterator(); while (itr.hasNext() swapRecords.size() SWAP_RECORD_POLL_SIZE) { -FlowFileRecord record = itr.next(); +final FlowFileRecord record = itr.next(); swapRecords.add(record); itr.remove(); } @@ -606,7 +606,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue { boolean isExpired; migrateSwapToActive(); -boolean queueFullAtStart = queueFullRef.get(); +final boolean queueFullAtStart = queueFullRef.get(); do { flowFile = this.activeQueue.poll(); @@ -794,9 +794,6 @@ public final class StandardFlowFileQueue implements FlowFileQueue { writeLock.lock(); try { migrateSwapToActive(); -if (activeQueue.isEmpty()) { -return Collections.emptyList(); -} final long expirationMillis = this.flowFileExpirationMillis.get(); final boolean queueFullAtStart = queueFullRef.get(); @@ -804,6 +801,13 @@ public final class StandardFlowFileQueue implements FlowFileQueue { final ListFlowFileRecord selectedFlowFiles = new ArrayList(); final ListFlowFileRecord unselected = new ArrayList(); +// the prefetch doesn't allow us to add records back. So when this method is used, +// if there are prefetched records, we have to requeue them into the active queue first. +final PreFetch prefetch = preFetchRef.get(); +if (prefetch != null) { +requeueExpiredPrefetch(prefetch); +} + while (true) { FlowFileRecord flowFile = this.activeQueue.poll(); if (flowFile == null) { @@ -970,7 +974,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue { boolean updated = false; do { -QueueSize queueSize = unacknowledgedSizeRef.get(); +final QueueSize queueSize = unacknowledgedSizeRef.get(); final QueueSize newSize = new QueueSize(queueSize.getObjectCount() + addToCount, queueSize.getByteCount() + addToSize); updated = unacknowledgedSizeRef.compareAndSet(queueSize, newSize); } while (!updated);
[2/2] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/548f6d08 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/548f6d08 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/548f6d08 Branch: refs/heads/develop Commit: 548f6d08352291c0e553a8d081732b16405609c2 Parents: a5d6f88 2d0bb1c Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 19 13:08:49 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jun 19 13:08:49 2015 -0400 -- .../src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java | 5 + 1 file changed, 5 insertions(+) --
incubator-nifi git commit: NIFI-129: Fixed bug that caused root group port's isTransmitting to return false, even when there is data being transferred
Repository: incubator-nifi Updated Branches: refs/heads/develop 66365b0e1 - e50b20c4c NIFI-129: Fixed bug that caused root group port's isTransmitting to return false, even when there is data being transferred Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/e50b20c4 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/e50b20c4 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/e50b20c4 Branch: refs/heads/develop Commit: e50b20c4ce513046b692154e370b0b1f87e21e0a Parents: 66365b0 Author: Mark Payne marka...@hotmail.com Authored: Thu Jun 18 08:26:49 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 18 09:03:54 2015 -0400 -- .../java/org/apache/nifi/remote/StandardRootGroupPort.java | 8 ++-- 1 file changed, 2 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e50b20c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java index 67f28d2..4bb1683 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java @@ -252,7 +252,7 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort @Override public boolean isValid() { -return (getConnectableType() == ConnectableType.INPUT_PORT) ? !getConnections(Relationship.ANONYMOUS).isEmpty() : true; +return getConnectableType() == ConnectableType.INPUT_PORT ? !getConnections(Relationship.ANONYMOUS).isEmpty() : true; } @Override @@ -275,14 +275,10 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort return false; } -if (processScheduler.getActiveThreadCount(this) 0) { +if (!requestQueue.isEmpty()) { return true; } -if (requestQueue.isEmpty()) { -return false; -} - requestLock.lock(); try { return !activeRequests.isEmpty();
[3/8] incubator-nifi git commit: NIFI-626 - query timeout added
NIFI-626 - query timeout added Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/d98757e4 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d98757e4 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d98757e4 Branch: refs/heads/develop Commit: d98757e4c79a0c036165e6278fc99909fc2ebcaf Parents: 14bcad2 Author: Toivo Adams toivo.ad...@gmail.com Authored: Sun May 31 16:43:49 2015 +0300 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 12:35:55 2015 -0400 -- .../nifi/processors/standard/ExecuteSQL.java| 15 +++- .../processors/standard/TestExecuteSQL.java | 25 2 files changed, 34 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d98757e4/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java index 37fd0f7..07e6fe9 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java @@ -40,7 +40,6 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.FlowFileAccessException; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; @@ -79,6 +78,16 @@ public class ExecuteSQL extends AbstractProcessor { .expressionLanguageSupported(true) .build(); +public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder() +.name(Max Wait Time) +.description(The maximum amount of time allowed for a running SQL select query ++ , zero means there is no limit. Max time less than 1 second will be equal to zero.) +.defaultValue(0 seconds) +.required(true) +.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) +.sensitive(false) +.build(); + private final ListPropertyDescriptor propDescriptors; public ExecuteSQL() { @@ -89,6 +98,7 @@ public class ExecuteSQL extends AbstractProcessor { ArrayListPropertyDescriptor pds = new ArrayList(); pds.add(DBCP_SERVICE); pds.add(SQL_SELECT_QUERY); +pds.add(QUERY_TIMEOUT); propDescriptors = Collections.unmodifiableList(pds); } @@ -113,6 +123,8 @@ public class ExecuteSQL extends AbstractProcessor { final DBCPService dbcpService = context.getProperty(DBCP_SERVICE).asControllerService(DBCPService.class); final String selectQuery = context.getProperty(SQL_SELECT_QUERY).evaluateAttributeExpressions(incoming).getValue(); +final Integer queryTimeout = context.getProperty(QUERY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue(); + final StopWatch stopWatch = new StopWatch(true); try { @@ -120,6 +132,7 @@ public class ExecuteSQL extends AbstractProcessor { try { final Statement st = con.createStatement(); try { + st.setQueryTimeout(queryTimeout); // timeout in seconds FlowFile outgoing = session.write(incoming, new OutputStreamCallback() { @Override public void process(final OutputStream out) throws IOException { http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d98757e4/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQL.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQL.java
[8/8] incubator-nifi git commit: Merge branch 'NIFI-626' into develop
Merge branch 'NIFI-626' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/0af9c75d Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/0af9c75d Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/0af9c75d Branch: refs/heads/develop Commit: 0af9c75d78c37f1551c53c0a4043d9f3fa5a82e3 Parents: a4d21e3 118b660 Author: Mark Payne marka...@hotmail.com Authored: Wed Jun 17 13:33:08 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 13:33:08 2015 -0400 -- .../nifi-standard-processors/pom.xml| 553 ++- .../nifi/processors/standard/ExecuteSQL.java| 157 ++ .../processors/standard/util/JdbcCommon.java| 165 ++ .../org.apache.nifi.processor.Processor | 1 + .../processors/standard/TestExecuteSQL.java | 169 ++ .../standard/util/TestJdbcCommon.java | 155 ++ .../standard/util/TestJdbcHugeStream.java | 267 + .../apache/nifi/dbcp/DBCPConnectionPool.java| 181 +++--- .../nifi/dbcp/DatabaseSystemDescriptor.java | 1 + .../org/apache/nifi/dbcp/DatabaseSystems.java | 1 + .../org/apache/nifi/dbcp/DBCPServiceTest.java | 160 ++ nifi/pom.xml| 10 + 12 files changed, 1344 insertions(+), 476 deletions(-) --
[4/8] incubator-nifi git commit: NIFI-626: Code cleanup to adhere to NiFi coding styles
NIFI-626: Code cleanup to adhere to NiFi coding styles Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a3b8e44a Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a3b8e44a Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a3b8e44a Branch: refs/heads/develop Commit: a3b8e44ad5aff7b68dde26c242ba171d1884b948 Parents: d98757e Author: Mark Payne marka...@hotmail.com Authored: Wed Jun 10 12:16:52 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 12:37:25 2015 -0400 -- .../nifi/processors/standard/ExecuteSQL.java| 71 +++-- .../processors/standard/util/JdbcCommon.java| 181 +++-- .../processors/standard/TestExecuteSQL.java | 57 ++-- .../standard/util/TestJdbcHugeStream.java | 259 +-- nifi/pom.xml| 5 + 5 files changed, 280 insertions(+), 293 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a3b8e44a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java index 07e6fe9..6647c4c 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java @@ -44,6 +44,7 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.standard.util.JdbcCommon; +import org.apache.nifi.util.LongHolder; import org.apache.nifi.util.StopWatch; @EventDriven @@ -62,7 +63,7 @@ public class ExecuteSQL extends AbstractProcessor { .description(SQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship) .build(); private final SetRelationship relationships; - + public static final PropertyDescriptor DBCP_SERVICE = new PropertyDescriptor.Builder() .name(Database Connection Pooling Service) .description(The Controller Service that is used to obtain connection to database) @@ -91,11 +92,11 @@ public class ExecuteSQL extends AbstractProcessor { private final ListPropertyDescriptor propDescriptors; public ExecuteSQL() { -HashSetRelationship r = new HashSet(); +final SetRelationship r = new HashSet(); r.add(REL_SUCCESS); relationships = Collections.unmodifiableSet(r); -ArrayListPropertyDescriptor pds = new ArrayList(); +final ListPropertyDescriptor pds = new ArrayList(); pds.add(DBCP_SERVICE); pds.add(SQL_SELECT_QUERY); pds.add(QUERY_TIMEOUT); @@ -113,7 +114,7 @@ public class ExecuteSQL extends AbstractProcessor { } @Override -public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { +public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { FlowFile incoming = session.get(); if (incoming == null) { return; @@ -126,44 +127,30 @@ public class ExecuteSQL extends AbstractProcessor { final Integer queryTimeout = context.getProperty(QUERY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue(); final StopWatch stopWatch = new StopWatch(true); - -try { - final Connection con = dbcpService.getConnection(); - try { - final Statement st = con.createStatement(); - try { - st.setQueryTimeout(queryTimeout); // timeout in seconds - FlowFile outgoing = session.write(incoming, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - try { - logger.info(start executing query {}, new Object[]{selectQuery}); -
[7/8] incubator-nifi git commit: NIFI-626: Fixed checkstyle violations
NIFI-626: Fixed checkstyle violations Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/118b6608 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/118b6608 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/118b6608 Branch: refs/heads/develop Commit: 118b6608c7334b577e4957653661ef4a488ebc6b Parents: ec02e58 Author: Mark Payne marka...@hotmail.com Authored: Wed Jun 17 13:26:22 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 13:31:02 2015 -0400 -- .../nifi-standard-processors/pom.xml| 8 + .../nifi/processors/standard/ExecuteSQL.java| 54 +++ .../processors/standard/util/JdbcCommon.java| 51 --- .../processors/standard/TestExecuteSQL.java | 67 + .../standard/util/TestJdbcCommon.java | 100 ++--- .../standard/util/TestJdbcHugeStream.java | 132 + .../apache/nifi/dbcp/DBCPConnectionPool.java| 146 ++- .../org/apache/nifi/dbcp/DBCPServiceTest.java | 122 +++- 8 files changed, 350 insertions(+), 330 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/118b6608/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml index 3f83160..aa8bfbe 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml @@ -182,6 +182,14 @@ groupIdorg.apache.nifi/groupId artifactIdnifi-dbcp-service-api/artifactId /dependency + + dependency + groupIdorg.apache.derby/groupId + artifactIdderby/artifactId + version10.11.1.1/version + scopetest/scope + /dependency + /dependencies build plugins http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/118b6608/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java index 6647c4c..bee1d39 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java @@ -48,41 +48,41 @@ import org.apache.nifi.util.LongHolder; import org.apache.nifi.util.StopWatch; @EventDriven -@Tags({sql, select, jdbc, query, database}) +@Tags({ sql, select, jdbc, query, database }) @CapabilityDescription(Execute provided SQL select query. Query result will be converted to Avro format. - + Streaming is used so arbitrarily large result sets are supported.) ++ Streaming is used so arbitrarily large result sets are supported.) public class ExecuteSQL extends AbstractProcessor { // Relationships public static final Relationship REL_SUCCESS = new Relationship.Builder() -.name(success) -.description(Successfully created FlowFile from SQL query result set.) -.build(); +.name(success) +.description(Successfully created FlowFile from SQL query result set.) +.build(); public static final Relationship REL_FAILURE = new Relationship.Builder() - .name(failure) - .description(SQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship) - .build(); +.name(failure) +.description(SQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship) +.build(); private final SetRelationship relationships; public static final PropertyDescriptor DBCP_SERVICE = new PropertyDescriptor.Builder() - .name(Database Connection Pooling Service) - .description(The Controller Service that is used to obtain connection to database) - .required(true) - .identifiesControllerService(DBCPService.class) -
[2/8] incubator-nifi git commit: NIFI-626 - First working version.
NIFI-626 - First working version. Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/14bcad21 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/14bcad21 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/14bcad21 Branch: refs/heads/develop Commit: 14bcad212a4e2c7242e892c0389c8da885e87e9c Parents: 13addeb Author: Toivo Adams toivo.ad...@gmail.com Authored: Tue May 26 14:36:59 2015 +0300 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 12:35:54 2015 -0400 -- .../nifi/processors/standard/ExecuteSQL.java| 77 + .../processors/standard/util/JdbcCommon.java| 13 +-- .../org.apache.nifi.processor.Processor | 1 + .../processors/standard/TestExecuteSQL.java | 91 +--- 4 files changed, 101 insertions(+), 81 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/14bcad21/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java index 9003b4a..37fd0f7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java @@ -17,7 +17,6 @@ package org.apache.nifi.processors.standard; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.sql.Connection; import java.sql.ResultSet; @@ -30,24 +29,20 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.dbcp.DBCPService; -import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.processor.AbstractProcessor; -import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.EventDriven; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.FlowFileAccessException; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; -import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.standard.util.JdbcCommon; import org.apache.nifi.util.StopWatch; @@ -92,6 +87,7 @@ public class ExecuteSQL extends AbstractProcessor { relationships = Collections.unmodifiableSet(r); ArrayListPropertyDescriptor pds = new ArrayList(); +pds.add(DBCP_SERVICE); pds.add(SQL_SELECT_QUERY); propDescriptors = Collections.unmodifiableList(pds); } @@ -108,47 +104,54 @@ public class ExecuteSQL extends AbstractProcessor { @Override public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { -FlowFile flowFile = session.get(); -if (flowFile == null) { +FlowFile incoming = session.get(); +if (incoming == null) { return; } final ProcessorLog logger = getLogger(); final DBCPService dbcpService = context.getProperty(DBCP_SERVICE).asControllerService(DBCPService.class); -final String selectQuery = context.getProperty(SQL_SELECT_QUERY).getValue(); +final String selectQuery = context.getProperty(SQL_SELECT_QUERY).evaluateAttributeExpressions(incoming).getValue(); + final StopWatch stopWatch = new StopWatch(true); try { final Connection con = dbcpService.getConnection(); - final Statement st = con.createStatement(); -
[6/8] incubator-nifi git commit: NIFI-626: SQL timestamps were being added to Avro schema as string but value was being added as java.sql.Date object, which caused exceptions to be thrown. Now call to
NIFI-626: SQL timestamps were being added to Avro schema as string but value was being added as java.sql.Date object, which caused exceptions to be thrown. Now call toString() on the Date object to put to the record. Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/ec02e58d Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/ec02e58d Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/ec02e58d Branch: refs/heads/develop Commit: ec02e58d3c0a8162a32fb5c51651accf1d961d1b Parents: ae9b40e Author: Mark Payne marka...@hotmail.com Authored: Wed Jun 17 12:32:57 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 12:37:27 2015 -0400 -- .../nifi/processors/standard/util/JdbcCommon.java | 16 +++- 1 file changed, 15 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ec02e58d/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java index 8dff244..beca304 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java @@ -52,7 +52,21 @@ public class JdbcCommon { while (rs.next()) { for (int i = 1; i = nrOfColumns; i++) { final Object value = rs.getObject(i); -rec.put(i - 1, value); + +// The different types that we support are numbers (int, +// long, double, float), as well +// as boolean values and Strings. Since Avro doesn't provide +// timestamp types, we want to +// convert those to Strings. So we will cast anything other +// than numbers or booleans to +// strings by using to toString() method. +if (value == null) { +rec.put(i - 1, null); +} else if (value instanceof Number || value instanceof Boolean) { +rec.put(i - 1, value); +} else { +rec.put(i - 1, value.toString()); +} } dataFileWriter.append(rec); nrOfRows += 1;
[5/8] incubator-nifi git commit: NIFI-677, DBCPService should allow user to specify Connection URL should fix also NIFI-676 and NIFI-678
NIFI-677, DBCPService should allow user to specify Connection URL should fix also NIFI-676 and NIFI-678 Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/ae9b40ec Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/ae9b40ec Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/ae9b40ec Branch: refs/heads/develop Commit: ae9b40ec6f71f69b7866d309464476686d3eebfa Parents: a3b8e44 Author: Toivo Adams toivo.ad...@gmail.com Authored: Thu Jun 11 18:27:23 2015 +0300 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 12:37:26 2015 -0400 -- .../apache/nifi/dbcp/DBCPConnectionPool.java| 51 -- .../nifi/dbcp/DatabaseSystemDescriptor.java | 1 + .../org/apache/nifi/dbcp/DatabaseSystems.java | 1 + .../org/apache/nifi/dbcp/DBCPServiceTest.java | 54 +--- 4 files changed, 25 insertions(+), 82 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ae9b40ec/nifi/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/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java index ef42be3..9ea93bb 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java @@ -49,35 +49,19 @@ import org.apache.nifi.reporting.InitializationException; @CapabilityDescription(Provides Database Connection Pooling Service. Connections can be asked from pool and returned after usage.) public class DBCPConnectionPool extends AbstractControllerService implements DBCPService { -public static final DatabaseSystemDescriptor DEFAULT_DATABASE_SYSTEM = DatabaseSystems.getDescriptor(JavaDB); - -public static final PropertyDescriptor DATABASE_SYSTEM = new PropertyDescriptor.Builder() -.name(Database Type) -.description(Database management system) -.allowableValues(DatabaseSystems.knownDatabaseSystems) -.defaultValue(DEFAULT_DATABASE_SYSTEM.getValue()) -.required(true) -.build(); - -public static final PropertyDescriptor DB_HOST = new PropertyDescriptor.Builder() -.name(Database Host) -.description(Database Host) +public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder() +.name(Database connection URL) +.description(A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters. ++ The exact syntax of a database connection URL is specified by your DBMS.) .defaultValue(null) -.required(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) -.build(); - -public static final PropertyDescriptor DB_PORT = new PropertyDescriptor.Builder() -.name(Database Port) -.description(Database server port) .required(true) -.addValidator(StandardValidators.PORT_VALIDATOR) .build(); public static final PropertyDescriptor DB_DRIVERNAME = new PropertyDescriptor.Builder() .name(Database Driver Class Name) .description(Database driver class name) -.defaultValue(DEFAULT_DATABASE_SYSTEM.driverClassName) +.defaultValue(null) .required(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .build(); @@ -90,14 +74,6 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC .addValidator(StandardValidators.URL_VALIDATOR) .build(); -public static final PropertyDescriptor DB_NAME = new PropertyDescriptor.Builder() -.name(Database Name) -.description(Database name) -.defaultValue(null) -.required(true) -.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) -.build(); - public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder() .name(Database User) .description(Database user name) @@ -110,9 +86,9 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC .name(Password) .description(The password for the database user) .defaultValue(null) -.required(true) -
[1/8] incubator-nifi git commit: NIFI-626 - ExecuteSQL processor for executing arbitrary SQL queries, initial implementation.
Repository: incubator-nifi Updated Branches: refs/heads/develop a4d21e3cd - 0af9c75d7 NIFI-626 - ExecuteSQL processor for executing arbitrary SQL queries, initial implementation. Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/13addeb5 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/13addeb5 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/13addeb5 Branch: refs/heads/develop Commit: 13addeb571746f8ace93928ab423af3ed1a3bc3c Parents: 69f04cb Author: Toivo Adams toivo.ad...@gmail.com Authored: Mon May 25 15:13:57 2015 +0300 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 12:35:53 2015 -0400 -- .../nifi-standard-processors/pom.xml| 545 ++- .../nifi/processors/standard/ExecuteSQL.java| 154 ++ .../processors/standard/util/JdbcCommon.java| 146 + .../processors/standard/TestExecuteSQL.java | 129 + .../standard/util/TestJdbcCommon.java | 153 ++ .../standard/util/TestJdbcHugeStream.java | 264 + nifi/pom.xml| 5 + 7 files changed, 1126 insertions(+), 270 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/13addeb5/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml index 8cce56d..3f83160 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml @@ -1,272 +1,277 @@ ?xml version=1.0? -!-- - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the License); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an AS IS BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. --- -project xmlns=http://maven.apache.org/POM/4.0.0; xmlns:xsi=http://www.w3.org/2001/XMLSchema-instance; xsi:schemaLocation=http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd; -modelVersion4.0.0/modelVersion -parent -groupIdorg.apache.nifi/groupId -artifactIdnifi-standard-bundle/artifactId -version0.2.0-incubating-SNAPSHOT/version -/parent -artifactIdnifi-standard-processors/artifactId -packagingjar/packaging -dependencies -dependency -groupIdorg.apache.nifi/groupId -artifactIdnifi-api/artifactId -scopeprovided/scope -/dependency -dependency -groupIdorg.apache.nifi/groupId -artifactIdnifi-processor-utils/artifactId -/dependency -dependency -groupIdorg.apache.nifi/groupId -artifactIdnifi-utils/artifactId -/dependency -dependency -groupIdorg.apache.nifi/groupId -artifactIdnifi-ssl-context-service-api/artifactId -/dependency -dependency -groupIdorg.apache.nifi/groupId -artifactIdnifi-flowfile-packager/artifactId -/dependency -dependency -groupIdorg.apache.nifi/groupId -artifactIdnifi-distributed-cache-client-service-api/artifactId -/dependency -dependency -groupIdorg.apache.nifi/groupId -artifactIdnifi-http-context-map-api/artifactId -/dependency -dependency -groupIdcommons-io/groupId -artifactIdcommons-io/artifactId -/dependency -dependency -groupIdcom.sun.jersey/groupId -artifactIdjersey-client/artifactId -/dependency -dependency -groupIdcom.sun.jersey/groupId -artifactIdjersey-server/artifactId -/dependency -dependency -groupIdcommons-net/groupId -artifactIdcommons-net/artifactId -/dependency -dependency -
incubator-nifi git commit: NIFI-37: Ensure that prov events that are emitted are emitted against FlowFiles that are known to that session.
Repository: incubator-nifi Updated Branches: refs/heads/develop 28abc9acb - 739baa2e5 NIFI-37: Ensure that prov events that are emitted are emitted against FlowFiles that are known to that session. Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/739baa2e Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/739baa2e Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/739baa2e Branch: refs/heads/develop Commit: 739baa2e5711b0677a451c4acadb10e232d5bf2b Parents: 28abc9a Author: Mark Payne marka...@hotmail.com Authored: Tue Jun 16 20:32:22 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 07:56:49 2015 -0400 -- .../apache/nifi/util/MockProcessSession.java| 40 +- .../nifi/util/MockProvenanceReporter.java | 365 --- .../apache/nifi/util/MockSessionFactory.java| 8 +- .../apache/nifi/util/SharedSessionState.java| 26 +- .../nifi/util/StandardProcessorTestRunner.java | 34 +- .../java/org/apache/nifi/util/TestRunner.java | 15 + .../repository/StandardProcessSession.java | 27 +- .../repository/StandardProvenanceReporter.java | 32 +- .../TestStandardProvenanceReporter.java | 7 +- .../nifi/processors/kafka/TestPutKafka.java | 52 +-- .../prioritizer/NewestFirstPrioritizerTest.java | 16 +- .../prioritizer/OldestFirstPrioritizerTest.java | 16 +- .../PriorityAttributePrioritizerTest.java | 27 +- 13 files changed, 499 insertions(+), 166 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/739baa2e/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java -- diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java index e9bb778..13198dc 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java @@ -40,11 +40,11 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.regex.Pattern; -import org.junit.Assert; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.FlowFileFilter; import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.QueueSize; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.FlowFileAccessException; @@ -54,6 +54,7 @@ import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.provenance.ProvenanceReporter; +import org.junit.Assert; public class MockProcessSession implements ProcessSession { @@ -65,14 +66,16 @@ public class MockProcessSession implements ProcessSession { private final MapLong, MockFlowFile originalVersions = new HashMap(); private final SharedSessionState sharedState; private final MapString, Long counterMap = new HashMap(); +private final ProvenanceReporter provenanceReporter; private boolean committed = false; private boolean rolledback = false; private int removedCount = 0; -public MockProcessSession(final SharedSessionState sharedState) { +public MockProcessSession(final SharedSessionState sharedState, final Processor processor) { this.sharedState = sharedState; this.processorQueue = sharedState.getFlowFileQueue(); +provenanceReporter = new MockProvenanceReporter(this, sharedState, processor.getIdentifier(), processor.getClass().getSimpleName()); } @Override @@ -194,7 +197,7 @@ public class MockProcessSession implements ProcessSession { try { out.write(mock.getData()); -} catch (IOException e) { +} catch (final IOException e) { throw new FlowFileAccessException(e.toString(), e); } } @@ -409,7 +412,7 @@ public class MockProcessSession implements ProcessSession { final ByteArrayInputStream bais = new ByteArrayInputStream(mock.getData()); try { callback.process(bais); -} catch (IOException e) { +} catch (final IOException e) { throw new ProcessException(e.toString(), e); } } @@ -766,7 +769,7 @@ public class MockProcessSession implements ProcessSession { if (source == null || destination == null || source == destination) { return destination; //don't need to inherit from ourselves } -
incubator-nifi git commit: NIFI-488: Merged develop
Repository: incubator-nifi Updated Branches: refs/heads/develop 315af02c5 - 69f04cbb8 NIFI-488: Merged develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/69f04cbb Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/69f04cbb Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/69f04cbb Branch: refs/heads/develop Commit: 69f04cbb8668b9c1500e23c2dbe41ebf97644bd0 Parents: 315af02 Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 8 14:13:24 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 08:19:36 2015 -0400 -- .../src/main/assembly/dependencies.xml | 2 + nifi/nifi-bootstrap/pom.xml | 7 + .../org/apache/nifi/bootstrap/NiFiListener.java | 13 +- .../java/org/apache/nifi/bootstrap/RunNiFi.java | 395 +++ .../org/apache/nifi/bootstrap/ShutdownHook.java | 6 +- .../src/main/resources/bin/dump-nifi.bat| 4 +- .../src/main/resources/bin/nifi.sh | 21 +- .../src/main/resources/bin/run-nifi.bat | 4 +- .../src/main/resources/bin/start-nifi.bat | 47 --- .../src/main/resources/bin/stop-nifi.bat| 47 --- .../src/main/resources/conf/logback.xml | 48 +++ 11 files changed, 320 insertions(+), 274 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/69f04cbb/nifi/nifi-assembly/src/main/assembly/dependencies.xml -- diff --git a/nifi/nifi-assembly/src/main/assembly/dependencies.xml b/nifi/nifi-assembly/src/main/assembly/dependencies.xml index 27eb32d..a3e3a18 100644 --- a/nifi/nifi-assembly/src/main/assembly/dependencies.xml +++ b/nifi/nifi-assembly/src/main/assembly/dependencies.xml @@ -49,6 +49,8 @@ useTransitiveFilteringtrue/useTransitiveFiltering includes includenifi-bootstrap/include + includeslf4j-api/include + includelogback-classic/include /includes /dependencySet http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/69f04cbb/nifi/nifi-bootstrap/pom.xml -- diff --git a/nifi/nifi-bootstrap/pom.xml b/nifi/nifi-bootstrap/pom.xml index bdf5089..ff27fd3 100644 --- a/nifi/nifi-bootstrap/pom.xml +++ b/nifi/nifi-bootstrap/pom.xml @@ -21,4 +21,11 @@ /parent artifactIdnifi-bootstrap/artifactId packagingjar/packaging + +dependencies + dependency + groupIdorg.slf4j/groupId + artifactIdslf4j-api/artifactId + /dependency +/dependencies /project http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/69f04cbb/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java -- diff --git a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java index 1713478..8d74f16 100644 --- a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java +++ b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java @@ -23,6 +23,7 @@ import java.net.ServerSocket; import java.net.Socket; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import org.apache.nifi.bootstrap.util.LimitingInputStream; @@ -40,6 +41,7 @@ public class NiFiListener { listener = new Listener(serverSocket, runner); final Thread listenThread = new Thread(listener); listenThread.setName(Listen to NiFi); +listenThread.setDaemon(true); listenThread.start(); return localPort; } @@ -62,7 +64,16 @@ public class NiFiListener { public Listener(final ServerSocket serverSocket, final RunNiFi runner) { this.serverSocket = serverSocket; -this.executor = Executors.newFixedThreadPool(2); +this.executor = Executors.newFixedThreadPool(2, new ThreadFactory() { +@Override +public Thread newThread(final Runnable runnable) { +final Thread t = Executors.defaultThreadFactory().newThread(runnable); +t.setDaemon(true); +t.setName(NiFi Bootstrap Command Listener); +return t; +} +}); + this.runner = runner; } http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/69f04cbb/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
incubator-nifi git commit: NIFI-454: Use random ports instead of specific ports for running unit tests; updated abstract class and interface to expose the port being used
Repository: incubator-nifi Updated Branches: refs/heads/develop 739baa2e5 - 315af02c5 NIFI-454: Use random ports instead of specific ports for running unit tests; updated abstract class and interface to expose the port being used Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/315af02c Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/315af02c Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/315af02c Branch: refs/heads/develop Commit: 315af02c595fdf62881b839f44227d121cbda46d Parents: 739baa2 Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 5 13:56:45 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 17 08:15:10 2015 -0400 -- .../cache/server/AbstractCacheServer.java | 8 +- .../distributed/cache/server/CacheServer.java | 2 + .../cache/server/DistributedCacheServer.java| 65 +++-- .../cache/server/TestServerAndClient.java | 292 +-- 4 files changed, 185 insertions(+), 182 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/315af02c/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java index 10f53b2..5c5a9cb 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java @@ -52,7 +52,6 @@ public abstract class AbstractCacheServer implements CacheServer { private final SSLContext sslContext; protected volatile boolean stopped = false; private final SetThread processInputThreads = new CopyOnWriteArraySet(); -; private volatile ServerSocketChannel serverSocketChannel; @@ -63,6 +62,11 @@ public abstract class AbstractCacheServer implements CacheServer { } @Override +public int getPort() { +return serverSocketChannel == null ? this.port : serverSocketChannel.socket().getLocalPort(); +} + +@Override public void start() throws IOException { serverSocketChannel = ServerSocketChannel.open(); serverSocketChannel.configureBlocking(true); @@ -117,7 +121,7 @@ public abstract class AbstractCacheServer implements CacheServer { return; } try (final InputStream in = new BufferedInputStream(rawInputStream); -final OutputStream out = new BufferedOutputStream(rawOutputStream)) { +final OutputStream out = new BufferedOutputStream(rawOutputStream)) { final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1); http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/315af02c/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java index fab8f13..d97c519 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java @@ -24,4 +24,6 @@ public interface CacheServer { void stop() throws IOException; +int getPort(); +
incubator-nifi git commit: Typo in documentation for EncryptContent
Repository: incubator-nifi Updated Branches: refs/heads/develop 86cbfab14 - 976cfaf60 Typo in documentation for EncryptContent Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/976cfaf6 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/976cfaf6 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/976cfaf6 Branch: refs/heads/develop Commit: 976cfaf60253e54b18248b7ee58b982a0749e9d1 Parents: 86cbfab Author: Jon Anderson andersoniii@gmail.com Authored: Tue May 19 20:55:59 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed Jun 10 10:41:18 2015 -0400 -- .../additionalDetails.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/976cfaf6/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/additionalDetails.html -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/additionalDetails.html index b0603c6..e6417b3 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/additionalDetails.html @@ -16,7 +16,7 @@ -- head meta charset=utf-8/ -titleEvaluateJsonPath/title +titleEncryptContent/title link rel=stylesheet href=../../css/component-usage.css type=text/css/ /head
[2/3] incubator-nifi git commit: NIFI-571: Removed checking ListListControllerServiceNode to see if it contains a ControllerServiceNode, as it would always return false; intent was to check all in
NIFI-571: Removed checking ListListControllerServiceNode to see if it contains a ControllerServiceNode, as it would always return false; intent was to check all inner Lists, but it's really not necessary (a bit of an optimization) that just confuses things Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/facc4242 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/facc4242 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/facc4242 Branch: refs/heads/develop Commit: facc4242bd1cf601fb2fc6b81745f7867c207616 Parents: 0898971 Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 5 16:01:09 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 8 20:20:34 2015 -0400 -- .../service/StandardControllerServiceProvider.java| 14 +- 1 file changed, 5 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/facc4242/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index d8506c1..f0de8d2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -357,7 +357,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi if (bulletinRepo != null) { bulletinRepo.addBulletin(BulletinFactory.createBulletin( -Controller Service, Severity.ERROR.name(), Could not start + serviceNode + due to + e)); +Controller Service, Severity.ERROR.name(), Could not start + serviceNode + due to + e)); } } } @@ -391,10 +391,6 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi final ListListControllerServiceNode orderedNodeLists = new ArrayList(); for (final ControllerServiceNode node : serviceNodeMap.values()) { -if (orderedNodeLists.contains(node)) { -continue; // this node is already in the list. -} - final ListControllerServiceNode branch = new ArrayList(); determineEnablingOrder(serviceNodeMap, node, branch, new HashSetControllerServiceNode()); orderedNodeLists.add(branch); @@ -404,10 +400,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } private static void determineEnablingOrder( -final MapString, ControllerServiceNode serviceNodeMap, -final ControllerServiceNode contextNode, -final ListControllerServiceNode orderedNodes, -final SetControllerServiceNode visited) { +final MapString, ControllerServiceNode serviceNodeMap, +final ControllerServiceNode contextNode, +final ListControllerServiceNode orderedNodes, +final SetControllerServiceNode visited) { if (visited.contains(contextNode)) { return; }
[1/3] incubator-nifi git commit: NIFI-662: Unit test expects \n when it should expect System.lineSeparator()
Repository: incubator-nifi Updated Branches: refs/heads/develop 0b4cddbe4 - b82d1428b NIFI-662: Unit test expects \n when it should expect System.lineSeparator() Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/08989719 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/08989719 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/08989719 Branch: refs/heads/develop Commit: 089897190c9b904701649a9e0cb1ad92e9549996 Parents: 096ca61 Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 5 15:58:41 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 8 20:20:33 2015 -0400 -- .../nifi/processors/standard/TestExecuteStreamCommand.java | 8 +--- 1 file changed, 1 insertion(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/08989719/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java index d565bca..9544105 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java @@ -22,8 +22,6 @@ import org.apache.nifi.util.TestRunners; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileInputStream; @@ -39,16 +37,12 @@ import static org.junit.Assert.assertTrue; * */ public class TestExecuteStreamCommand { - -private static Logger LOGGER; - @BeforeClass public static void init() { System.setProperty(org.slf4j.simpleLogger.defaultLogLevel, info); System.setProperty(org.slf4j.simpleLogger.showDateTime, true); System.setProperty(org.slf4j.simpleLogger.log.nifi.processors.standard.ExecuteStreamCommand, debug); System.setProperty(org.slf4j.simpleLogger.log.nifi.processors.standard.TestExecuteStreamCommand, debug); -LOGGER = LoggerFactory.getLogger(TestExecuteStreamCommand.class); } @Test @@ -177,7 +171,7 @@ public class TestExecuteStreamCommand { byte[] byteArray = flowFiles.get(0).toByteArray(); String result = new String(byteArray); assertTrue(TestIngestAndUpdate.jar should not have received anything to modify, - Pattern.compile(target:ModifiedResult\r?\n$).matcher(result).find()); + Pattern.compile(target:ModifiedResult\r?\n$).matcher(result).find()); } // this is dependent on window with cygwin...so it's not enabled
[3/3] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/b82d1428 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b82d1428 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b82d1428 Branch: refs/heads/develop Commit: b82d1428b24b40fc2f003655979969632e0f5b7f Parents: facc424 0b4cddb Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 8 20:22:48 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Mon Jun 8 20:22:48 2015 -0400 -- .../processors/standard/TestExecuteStreamCommand.java | 12 1 file changed, 8 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b82d1428/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java --
incubator-nifi git commit: NIFI-625: Fixed bug in expression language that caused EL not to get evaluated if enclosed within curly braces
Repository: incubator-nifi Updated Branches: refs/heads/develop b82d1428b - 86cbfab14 NIFI-625: Fixed bug in expression language that caused EL not to get evaluated if enclosed within curly braces Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/86cbfab1 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/86cbfab1 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/86cbfab1 Branch: refs/heads/develop Commit: 86cbfab14ad13ce5adbb591deae9c8d46fd7274e Parents: b82d142 Author: Mark Payne marka...@hotmail.com Authored: Mon Jun 8 19:52:15 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Tue Jun 9 18:32:34 2015 -0400 -- .../attribute/expression/language/Query.java| 135 ++- .../expression/language/TestQuery.java | 54 +--- .../processors/standard/TestReplaceText.java| 18 +++ 3 files changed, 123 insertions(+), 84 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/86cbfab1/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java -- diff --git a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java index 7e40897..77d08c6 100644 --- a/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java +++ b/nifi/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java @@ -244,51 +244,58 @@ public class Query { int backslashCount = 0; charLoop: -for (int i = 0; i value.length(); i++) { -final char c = value.charAt(i); - -if (expressionStart -1 (c == '\'' || c == '') (lastChar != '\\' || backslashCount % 2 == 0)) { -final int endQuoteIndex = findEndQuoteChar(value, i); -if (endQuoteIndex 0) { -break charLoop; -} - -i = endQuoteIndex; -continue; -} +for (int i = 0; i value.length(); i++) { +final char c = value.charAt(i); -if (c == '{') { -if (oddDollarCount lastChar == '$') { -if (embeddedCount == 0) { -expressionStart = i - 1; +if (expressionStart -1 (c == '\'' || c == '') (lastChar != '\\' || backslashCount % 2 == 0)) { +final int endQuoteIndex = findEndQuoteChar(value, i); +if (endQuoteIndex 0) { +break charLoop; } -} -embeddedCount++; -} else if (c == '}') { -if (embeddedCount = 0) { +i = endQuoteIndex; continue; } -if (--embeddedCount == 0) { +if (c == '{') { +if (oddDollarCount lastChar == '$') { +if (embeddedCount == 0) { +expressionStart = i - 1; +} +} + +// Keep track of the number of opening curly braces that we are embedded within, +// if we are within an Expression. If we are outside of an Expression, we can just ignore +// curly braces. This allows us to ignore the first character if the value is something +// like: { ${abc} } +// However, we will count the curly braces if we have something like: ${ $${abc} } if (expressionStart -1) { -// ended expression. Add a new range. -final Range range = new Range(expressionStart, i); -ranges.add(range); +embeddedCount++; +} +} else if (c == '}') { +if (embeddedCount = 0) { +continue; } -expressionStart = -1; +if (--embeddedCount == 0) { +if (expressionStart -1) { +// ended expression. Add a new range. +final Range range = new Range(expressionStart, i); +ranges.add(range); +} + +expressionStart = -1; +} +} else if (c == '$') { +
[1/2] incubator-nifi git commit: Make Windows batch scripts a bit more user friendly. Do no move out of the current working directory and try a little harder to find Java.
Repository: incubator-nifi Updated Branches: refs/heads/develop 5fb5fa6da - ceda66159 Make Windows batch scripts a bit more user friendly. Do no move out of the current working directory and try a little harder to find Java. Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f1607282 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f1607282 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f1607282 Branch: refs/heads/develop Commit: f160728253150e690408b760286a11e8adbb88eb Parents: 82174e4 Author: BELUGABEHR belugab...@users.noreply.github.com Authored: Tue May 26 21:52:08 2015 -0400 Committer: BELUGABEHR belugab...@users.noreply.github.com Committed: Tue May 26 21:52:08 2015 -0400 -- .../src/main/resources/bin/dump-nifi.bat| 80 .../src/main/resources/bin/nifi-status.bat | 33 .../src/main/resources/bin/run-nifi.bat | 80 .../src/main/resources/bin/start-nifi.bat | 80 .../src/main/resources/bin/status-nifi.bat | 47 .../src/main/resources/bin/stop-nifi.bat| 80 6 files changed, 235 insertions(+), 165 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f1607282/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/dump-nifi.bat -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/dump-nifi.bat b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/dump-nifi.bat index 71e5a1a..ab902d0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/dump-nifi.bat +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/dump-nifi.bat @@ -1,33 +1,47 @@ -@echo off -rem -remLicensed to the Apache Software Foundation (ASF) under one or more -remcontributor license agreements. See the NOTICE file distributed with -remthis work for additional information regarding copyright ownership. -remThe ASF licenses this file to You under the Apache License, Version 2.0 -rem(the License); you may not use this file except in compliance with -remthe License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -remUnless required by applicable law or agreed to in writing, software -remdistributed under the License is distributed on an AS IS BASIS, -remWITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -remSee the License for the specific language governing permissions and -remlimitations under the License. -rem - -rem Use JAVA_HOME if it's set; otherwise, just use java -IF %JAVA_HOME%== (SET JAVA_EXE=java) ELSE (SET JAVA_EXE=%JAVA_HOME%\bin\java.exe) - -SET NIFI_ROOT=%~dp0..\ -CD /d %NIFI_ROOT% -SET LIB_DIR=lib\bootstrap -SET CONF_DIR=conf - -SET BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf -SET JAVA_ARGS=-Dorg.apache.nifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE% - -SET JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi -SET BOOTSTRAP_ACTION=dump - -cmd.exe /C %JAVA_EXE% %JAVA_PARAMS% %BOOTSTRAP_ACTION% +@echo off +rem +remLicensed to the Apache Software Foundation (ASF) under one or more +remcontributor license agreements. See the NOTICE file distributed with +remthis work for additional information regarding copyright ownership. +remThe ASF licenses this file to You under the Apache License, Version 2.0 +rem(the License); you may not use this file except in compliance with +remthe License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +remUnless required by applicable law or agreed to in writing, software +remdistributed under the License is distributed on an AS IS BASIS, +remWITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +remSee the License for the specific language governing permissions and +remlimitations under the License. +rem + +rem Use JAVA_HOME if it's set; otherwise, just use java + +if %JAVA_HOME% == goto noJavaHome +if not exist %JAVA_HOME%\bin\java.exe goto noJavaHome +set JAVA_EXE=%JAVA_HOME%\bin\java.exe +goto startNifi + +:noJavaHome +echo The JAVA_HOME environment variable is not defined correctly. +echo Instead the PATH will be used to find the java executable. +echo. +set JAVA_EXE=java +goto startNifi + +:startNifi +set NIFI_ROOT=%~dp0..\ +pushd %NIFI_ROOT% +set LIB_DIR=lib\bootstrap
[2/2] incubator-nifi git commit: Merge branch 'ms-batch' of https://github.com/BELUGABEHR/incubator-nifi into develop
Merge branch 'ms-batch' of https://github.com/BELUGABEHR/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/ceda6615 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/ceda6615 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/ceda6615 Branch: refs/heads/develop Commit: ceda66159e6b73f6abaf03bf580e2878624c23ad Parents: 5fb5fa6 f160728 Author: Mark Payne marka...@hotmail.com Authored: Sun Jun 7 15:02:01 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sun Jun 7 15:02:01 2015 -0400 -- .../src/main/resources/bin/dump-nifi.bat| 80 .../src/main/resources/bin/nifi-status.bat | 33 .../src/main/resources/bin/run-nifi.bat | 80 .../src/main/resources/bin/start-nifi.bat | 80 .../src/main/resources/bin/status-nifi.bat | 47 .../src/main/resources/bin/stop-nifi.bat| 80 6 files changed, 235 insertions(+), 165 deletions(-) --
[3/3] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/11bbd9cb Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/11bbd9cb Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/11bbd9cb Branch: refs/heads/develop Commit: 11bbd9cb42b1a61f3cf44b46e731d3eff15c1c90 Parents: 68be759 b9b628d Author: Mark Payne marka...@hotmail.com Authored: Sun Jun 7 08:09:00 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sun Jun 7 08:09:00 2015 -0400 -- nifi/nifi-assembly/LICENSE | 37 ++- nifi/nifi-assembly/NOTICE | 120 -- .../src/main/resources/META-INF/NOTICE | 27 ++- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../nifi-hadoop-libraries-nar/pom.xml | 96 .../src/main/resources/META-INF/LICENSE | 28 +++ .../src/main/resources/META-INF/NOTICE | 81 ++- .../src/main/resources/META-INF/LICENSE | 238 ++ .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/LICENSE | 202 .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 42 .../src/main/resources/META-INF/NOTICE | 24 ++ .../nifi-mongodb-processors/pom.xml | 1 - .../src/main/resources/META-INF/LICENSE | 113 + .../src/main/resources/META-INF/NOTICE | 13 +- .../src/main/resources/META-INF/LICENSE | 2 +- .../src/main/resources/META-INF/NOTICE | 50 +++- .../src/main/resources/META-INF/LICENSE | 239 +++ .../src/main/resources/META-INF/NOTICE | 13 + .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- nifi/pom.xml| 9 +- 26 files changed, 907 insertions(+), 444 deletions(-) --
[1/3] incubator-nifi git commit: NIFI-591: Recognize both application/bzip2 and application/x-bzip2 as bzip mime types but use application/x-bzip2 when compressing
Repository: incubator-nifi Updated Branches: refs/heads/develop b9b628dec - 11bbd9cb4 NIFI-591: Recognize both application/bzip2 and application/x-bzip2 as bzip mime types but use application/x-bzip2 when compressing Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/83785a97 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/83785a97 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/83785a97 Branch: refs/heads/develop Commit: 83785a97989b31180a5925060604a27084dc62cf Parents: 8d1536e Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 5 14:48:39 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sun Jun 7 08:03:07 2015 -0400 -- .../processors/standard/CompressContent.java| 79 ++-- .../standard/TestCompressContent.java | 50 + 2 files changed, 90 insertions(+), 39 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/83785a97/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java index 585902b..1b9b20c 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java @@ -68,11 +68,11 @@ import org.tukaani.xz.XZOutputStream; @SupportsBatching @Tags({content, compress, decompress, gzip, bzip2, lzma, xz-lzma2}) @CapabilityDescription(Compresses or decompresses the contents of FlowFiles using a user-specified compression algorithm and updates the mime.type -+ attribute as appropriate) ++ attribute as appropriate) @ReadsAttribute(attribute = mime.type, description = If the Compression Format is set to use mime.type attribute, this attribute is used to -+ determine the compression type. Otherwise, this attribute is ignored.) ++ determine the compression type. Otherwise, this attribute is ignored.) @WritesAttribute(attribute = mime.type, description = If the Mode property is set to compress, the appropriate MIME Type is set. If the Mode -+ property is set to decompress and the file is successfully decompressed, this attribute is removed, as the MIME Type is no longer known.) ++ property is set to decompress and the file is successfully decompressed, this attribute is removed, as the MIME Type is no longer known.) public class CompressContent extends AbstractProcessor { public static final String COMPRESSION_FORMAT_ATTRIBUTE = use mime.type attribute; @@ -85,44 +85,44 @@ public class CompressContent extends AbstractProcessor { public static final String MODE_DECOMPRESS = decompress; public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder() -.name(Compression Format) -.description(The compression format to use. Valid values are: GZIP, BZIP2, XZ-LZMA2, and LZMA) -.allowableValues(COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_BZIP2, COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA) -.defaultValue(COMPRESSION_FORMAT_ATTRIBUTE) -.required(true) -.build(); +.name(Compression Format) +.description(The compression format to use. Valid values are: GZIP, BZIP2, XZ-LZMA2, and LZMA) +.allowableValues(COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_BZIP2, COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA) +.defaultValue(COMPRESSION_FORMAT_ATTRIBUTE) +.required(true) +.build(); public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder() -.name(Compression Level) -.description(The compression level to use; this is valid only when using GZIP compression. A lower value results in faster processing -+ but less compression; a value of 0 indicates no compression but simply archiving) -.defaultValue(1) -.required(true) -.allowableValues(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) -.build(); +.name(Compression Level) +.description(The compression level to use; this is valid only when using GZIP compression. A lower value results in
[2/3] incubator-nifi git commit: Merge branch 'NIFI-591' into develop
Merge branch 'NIFI-591' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/68be759f Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/68be759f Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/68be759f Branch: refs/heads/develop Commit: 68be759fe48e194fe5b2966512057b54a67b08a5 Parents: 56c173f 83785a9 Author: Mark Payne marka...@hotmail.com Authored: Sun Jun 7 08:08:47 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sun Jun 7 08:08:47 2015 -0400 -- .../processors/standard/CompressContent.java| 79 ++-- .../standard/TestCompressContent.java | 50 + 2 files changed, 90 insertions(+), 39 deletions(-) --
[4/4] incubator-nifi git commit: Merge branch 'NIFI-427' into develop
Merge branch 'NIFI-427' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/56c173fb Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/56c173fb Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/56c173fb Branch: refs/heads/develop Commit: 56c173fbd8df7ab6001a5e2f0480e78e00d73644 Parents: 35308ed 5c1afc0 Author: Mark Payne marka...@hotmail.com Authored: Sat Jun 6 20:24:05 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sat Jun 6 20:24:05 2015 -0400 -- .../org/apache/nifi/groups/ProcessGroup.java| 11 +++ .../nifi/groups/StandardProcessGroup.java | 88 ++-- 2 files changed, 75 insertions(+), 24 deletions(-) --
[2/4] incubator-nifi git commit: NIFI-663: Updating tests to provide valid options for assertions when in a Windows environment. Preferring regex to contains/endsWith
NIFI-663: Updating tests to provide valid options for assertions when in a Windows environment. Preferring regex to contains/endsWith Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/ffc5cdb9 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/ffc5cdb9 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/ffc5cdb9 Branch: refs/heads/develop Commit: ffc5cdb92c2d4b1e37645ad0a7f4ac18233de5d8 Parents: 63bdda4 Author: Aldrin Piri ald...@apache.org Authored: Sat Jun 6 11:16:52 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sat Jun 6 11:28:44 2015 -0400 -- .../standard/TestExecuteStreamCommand.java | 34 ++-- 1 file changed, 17 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ffc5cdb9/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java index 90fb28f..d565bca 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteStreamCommand.java @@ -16,25 +16,25 @@ */ package org.apache.nifi.processors.standard; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.List; - import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; - import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.regex.Pattern; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + /** * */ @@ -70,7 +70,7 @@ public class TestExecuteStreamCommand { MockFlowFile outputFlowFile = flowFiles.get(0); byte[] byteArray = outputFlowFile.toByteArray(); String result = new String(byteArray); -assertTrue(Test was a success\r\n.equals(result) || Test was a success\n.equals(result)); +assertTrue(Pattern.compile(Test was a success\r?\n).matcher(result).find()); assertEquals(0, outputFlowFile.getAttribute(execution.status)); assertEquals(java, outputFlowFile.getAttribute(execution.command)); assertEquals(-jar;, outputFlowFile.getAttribute(execution.command.args).substring(0, 5)); @@ -131,8 +131,7 @@ public class TestExecuteStreamCommand { byte[] byteArray = flowFiles.get(0).toByteArray(); String result = new String(byteArray); -assertTrue(result.contains(File.separator + nifi-standard-processors:ModifiedResult\r\n) - || result.contains(File.separator + nifi-standard-processors:ModifiedResult\n)); + assertTrue(Pattern.compile(nifi-standard-processors:ModifiedResult\r?\n).matcher(result).find()); } @Test @@ -153,8 +152,9 @@ public class TestExecuteStreamCommand { ListMockFlowFile flowFiles = controller.getFlowFilesForRelationship(ExecuteStreamCommand.OUTPUT_STREAM_RELATIONSHIP); byte[] byteArray = flowFiles.get(0).toByteArray(); String result = new String(byteArray); -assertTrue(result.contains(File.separator + nifi-standard-processors + File.separator + target:ModifiedResult\r\n) - || result.contains(File.separator + nifi-standard-processors + File.separator + target:ModifiedResult\n)); + +final String quotedSeparator = Pattern.quote(File.separator); +assertTrue(Pattern.compile(quotedSeparator + nifi-standard-processors + quotedSeparator + target:ModifiedResult\r?\n).matcher(result).find()); } @Test @@ -177,7 +177,7 @@ public class TestExecuteStreamCommand { byte[] byteArray = flowFiles.get(0).toByteArray(); String result = new String(byteArray); assertTrue(TestIngestAndUpdate.jar should not have received anything to
[1/4] incubator-nifi git commit: NIFI-427: Allow Process Groups to be deleted as long as no incoming or outgoing connections (unless they are also selected), all components stopped, no FlowFiles queue
Repository: incubator-nifi Updated Branches: refs/heads/develop c9ddf21ad - 56c173fbd NIFI-427: Allow Process Groups to be deleted as long as no incoming or outgoing connections (unless they are also selected), all components stopped, no FlowFiles queued Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5c1afc08 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5c1afc08 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5c1afc08 Branch: refs/heads/develop Commit: 5c1afc0897a939bcde803c07216be91b6c4d0c03 Parents: 8d1536e Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 5 12:59:15 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jun 5 16:17:50 2015 -0400 -- .../org/apache/nifi/groups/ProcessGroup.java| 11 +++ .../nifi/groups/StandardProcessGroup.java | 88 ++-- 2 files changed, 75 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5c1afc08/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java index a9cfb58..0e2a3f9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java @@ -641,6 +641,17 @@ public interface ProcessGroup { void verifyCanDelete(); +/** + * Ensures that the ProcessGroup is eligible to be deleted. + * + * @param ignorePortConnections if true, the Connections that are currently connected to Ports + * will be ignored. Otherwise, the ProcessGroup is not eligible for deletion if its input ports + * or output ports have any connections + * + * @throws IllegalStateException if the ProcessGroup is not eligible for deletion + */ +void verifyCanDelete(boolean ignorePortConnections); + void verifyCanStart(); void verifyCanStop(); http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5c1afc08/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 9fbfb5d..07e3e91 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -243,7 +243,7 @@ public final class StandardProcessGroup implements ProcessGroup { } return new ProcessGroupCounts(inputPortCount, outputPortCount, running, stopped, -invalid, disabled, activeRemotePorts, inactiveRemotePorts); +invalid, disabled, activeRemotePorts, inactiveRemotePorts); } @Override @@ -548,9 +548,7 @@ public final class StandardProcessGroup implements ProcessGroup { @Override public void removeProcessGroup(final ProcessGroup group) { -if (!requireNonNull(group).isEmpty()) { -throw new IllegalStateException(Cannot remove + group + because it is not empty); -} +requireNonNull(group).verifyCanDelete(); writeLock.lock(); try { @@ -558,7 +556,7 @@ public final class StandardProcessGroup implements ProcessGroup { if (toRemove == null) { throw new IllegalStateException(group + is not a member of this Process Group); } -verifyCanRemove(toRemove); +toRemove.verifyCanDelete(); processGroups.remove(group.getIdentifier()); @@ -568,12 +566,6 @@ public final class StandardProcessGroup implements ProcessGroup { } } -private void verifyCanRemove(final ProcessGroup childGroup) { -if (!childGroup.isEmpty()) { -throw new
[3/4] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/35308edf Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/35308edf Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/35308edf Branch: refs/heads/develop Commit: 35308edf4ef9a996af3c3b502353cdcddc3679b4 Parents: ffc5cdb c9ddf21 Author: Mark Payne marka...@hotmail.com Authored: Sat Jun 6 20:22:48 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Sat Jun 6 20:22:48 2015 -0400 -- .../nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js | 8 1 file changed, 4 insertions(+), 4 deletions(-) --
[2/3] incubator-nifi git commit: NIFI-41: Don't allow destination fo connection to change if current destination is holding its FlowFiles
NIFI-41: Don't allow destination fo connection to change if current destination is holding its FlowFiles Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/8d1536ed Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/8d1536ed Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/8d1536ed Branch: refs/heads/develop Commit: 8d1536ed24ea0b149f9fa14ead073b8decdead7d Parents: 66f3b7e Author: Mark Payne marka...@hotmail.com Authored: Fri Jun 5 08:49:01 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Fri Jun 5 08:49:01 2015 -0400 -- .../java/org/apache/nifi/controller/FlowFileQueue.java | 9 + .../apache/nifi/controller/StandardFlowFileQueue.java | 13 + .../apache/nifi/connectable/StandardConnection.java| 4 3 files changed, 22 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8d1536ed/nifi/nifi-api/src/main/java/org/apache/nifi/controller/FlowFileQueue.java -- diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/FlowFileQueue.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/FlowFileQueue.java index 92a4ee0..e1baeb7 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/FlowFileQueue.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/FlowFileQueue.java @@ -100,6 +100,15 @@ public interface FlowFileQueue { QueueSize getActiveQueueSize(); +/** + * Returns a QueueSize that represents all FlowFiles that are 'unacknowledged'. A FlowFile + * is considered to be unacknowledged if it has been pulled from the queue by some component + * but the session that pulled the FlowFile has not yet been committed or rolled back. + * + * @return a QueueSize that represents all FlowFiles that are 'unacknowledged'. + */ +QueueSize getUnacknowledgedQueueSize(); + void acknowledge(FlowFileRecord flowFile); void acknowledge(CollectionFlowFileRecord flowFiles); http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8d1536ed/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java -- diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java index 075f2cf..8f6c8ed 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java @@ -229,7 +229,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue { } return new QueueSize(activeQueue.size() + swappedRecordCount + unacknowledged.getObjectCount() + preFetchCount, -activeQueueContentSize + swappedContentSize + unacknowledged.getByteCount() + preFetchSize); +activeQueueContentSize + swappedContentSize + unacknowledged.getByteCount() + preFetchSize); } @Override @@ -526,9 +526,9 @@ public final class StandardFlowFileQueue implements FlowFileQueue { final QueueSize unacknowledged = unacknowledgedSizeRef.get(); logger.debug(Total Queue Size: ActiveQueue={}/{} MB, Swap Queue={}/{} MB, Unacknowledged={}/{} MB, -activeQueue.size(), activeQueueContentSize / byteToMbDivisor, -swappedRecordCount, swappedContentSize / byteToMbDivisor, -unacknowledged.getObjectCount(), unacknowledged.getByteCount() / byteToMbDivisor); +activeQueue.size(), activeQueueContentSize / byteToMbDivisor, +swappedRecordCount, swappedContentSize / byteToMbDivisor, +unacknowledged.getObjectCount(), unacknowledged.getByteCount() / byteToMbDivisor); } return swapQueue.size(); @@ -961,6 +961,11 @@ public final class StandardFlowFileQueue implements FlowFileQueue { writeLock.unlock(external unlock); } +@Override +public QueueSize getUnacknowledgedQueueSize() { +return unacknowledgedSizeRef.get(); +} + private void updateUnacknowledgedSize(final int addToCount, final long addToSize) { boolean updated = false;
[1/3] incubator-nifi git commit: NIFI-602: Ensure we read all data from socket after sending 'SHUTDOWN' command
Repository: incubator-nifi Updated Branches: refs/heads/develop 73b08d820 - 63bdda4b5 NIFI-602: Ensure we read all data from socket after sending 'SHUTDOWN' command Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/66f3b7e3 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/66f3b7e3 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/66f3b7e3 Branch: refs/heads/develop Commit: 66f3b7e30f7726308ac25b7d0f2a2856880d1852 Parents: efc862e Author: Mark Payne marka...@hotmail.com Authored: Thu Jun 4 14:10:43 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 4 14:10:43 2015 -0400 -- .../java/org/apache/nifi/bootstrap/RunNiFi.java | 34 1 file changed, 20 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/66f3b7e3/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java -- diff --git a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java index bb83e3d..e6b1bc5 100644 --- a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java +++ b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java @@ -229,7 +229,10 @@ public class RunNiFi { props.load(fis); } -logger.log(Level.FINE, Properties: {0}, props); +final MapObject, Object modified = new HashMap(props); +modified.remove(secret.key); +logger.log(Level.FINE, Properties: {0}, modified); + return props; } @@ -250,8 +253,8 @@ public class RunNiFi { Files.setPosixFilePermissions(statusFile.toPath(), perms); } catch (final Exception e) { logger.log(Level.WARNING, Failed to set permissions so that only the owner can read status file {0}; -+ this may allows others to have access to the key needed to communicate with NiFi. -+ Permissions should be changed so that only the owner can read this file, statusFile); ++ this may allows others to have access to the key needed to communicate with NiFi. ++ Permissions should be changed so that only the owner can read this file, statusFile); } try (final FileOutputStream fos = new FileOutputStream(statusFile)) { @@ -328,8 +331,8 @@ public class RunNiFi { boolean running = false; String line; try (final InputStream in = proc.getInputStream(); -final Reader streamReader = new InputStreamReader(in); -final BufferedReader reader = new BufferedReader(streamReader)) { +final Reader streamReader = new InputStreamReader(in); +final BufferedReader reader = new BufferedReader(streamReader)) { while ((line = reader.readLine()) != null) { if (line.trim().startsWith(pid)) { @@ -395,7 +398,7 @@ public class RunNiFi { final Status status = getStatus(); if (status.isRespondingToPing()) { logger.log(Level.INFO, Apache NiFi is currently running, listening to Bootstrap on port {0}, PID={1}, -new Object[]{status.getPort(), status.getPid() == null ? unknkown : status.getPid()}); +new Object[]{status.getPort(), status.getPid() == null ? unknkown : status.getPid()}); return; } @@ -488,12 +491,15 @@ public class RunNiFi { final OutputStream out = socket.getOutputStream(); out.write((SHUTDOWN_CMD + + secretKey + \n).getBytes(StandardCharsets.UTF_8)); out.flush(); -out.close(); +socket.shutdownOutput(); final InputStream in = socket.getInputStream(); -final BufferedReader reader = new BufferedReader(new InputStreamReader(in)); -final String response = reader.readLine(); -reader.close(); +int lastChar; +final StringBuilder sb = new StringBuilder(); +while ((lastChar = in.read()) -1) { +sb.append((char) lastChar); +} +final String response = sb.toString().trim(); logger.log(Level.FINE, Received response to SHUTDOWN command: {0}, response); @@ -557,7 +563,7 @@ public class RunNiFi { final Process proc = Runtime.getRuntime().exec(new String[]{ps, -o, pid, --no-headers, --ppid, ppid}); final ListString childPids = new ArrayList(); try (final InputStream in = proc.getInputStream(); -final BufferedReader reader = new
[2/3] incubator-nifi git commit: NIFI-657: Ignored Unit tests and removed maven plugin for embedding mongodb, as they fail in certain environments
NIFI-657: Ignored Unit tests and removed maven plugin for embedding mongodb, as they fail in certain environments Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/fcadaa89 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/fcadaa89 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/fcadaa89 Branch: refs/heads/develop Commit: fcadaa894b39318d11fa6dffa7846d87c9a50d8c Parents: 9fbd88f Author: Mark Payne marka...@hotmail.com Authored: Thu Jun 4 13:01:23 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 4 13:01:23 2015 -0400 -- .../nifi-mongodb-processors/pom.xml | 31 .../nifi/processors/mongodb/GetMongoTest.java | 4 ++- .../nifi/processors/mongodb/PutMongoTest.java | 8 +++-- 3 files changed, 8 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fcadaa89/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/pom.xml index db1c498..020f4dc 100644 --- a/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/pom.xml @@ -64,35 +64,4 @@ scopetest/scope /dependency /dependencies - -build -plugins -plugin -groupIdcom.github.joelittlejohn.embedmongo/groupId -artifactIdembedmongo-maven-plugin/artifactId -version0.1.12/version -executions -execution -idstart/id -goals -goalstart/goal -/goals -phasetest-compile/phase -configuration - databaseDirectory${project.build.directory}/embedmongo/db/databaseDirectory -loggingfile/logging - logFile${project.build.directory}/embedmongo.log/logFile -/configuration -/execution -execution -idstop/id -goals -goalstop/goal -/goals -phaseprepare-package/phase -/execution -/executions -/plugin -/plugins -/build /project http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fcadaa89/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoTest.java -- diff --git a/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoTest.java b/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoTest.java index 53f47f2..810fc4d 100644 --- a/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoTest.java +++ b/nifi/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/GetMongoTest.java @@ -33,6 +33,7 @@ import org.bson.Document; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import com.google.common.collect.Lists; @@ -40,6 +41,7 @@ import com.mongodb.MongoClient; import com.mongodb.MongoClientURI; import com.mongodb.client.MongoCollection; +@Ignore(Integration tests that cause failures in some environments. Require that they be run from Maven to run the embedded mongo maven plugin. Maven Plugin also fails in my CentOS 7 environment.) public class GetMongoTest { private static final String MONGO_URI = mongodb://localhost; private static final String DB_NAME = GetMongoTest.class.getSimpleName().toLowerCase(); @@ -49,7 +51,7 @@ public class GetMongoTest { new Document(_id, doc_1).append(a, 1).append(b, 2).append(c, 3), new Document(_id, doc_2).append(a, 1).append(b, 2).append(c, 4), new Document(_id, doc_3).append(a, 1).append(b, 3) -); +); private TestRunner runner; private MongoClient mongoClient;
[3/3] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/efc862ea Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/efc862ea Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/efc862ea Branch: refs/heads/develop Commit: efc862eac8f24058916b6c37c1674f2aa9577e01 Parents: fcadaa8 fb75a4b Author: Mark Payne marka...@hotmail.com Authored: Thu Jun 4 13:09:07 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Thu Jun 4 13:09:07 2015 -0400 -- .../nifi-hl7-query-language/pom.xml | 2 +- nifi/nifi-docs/pom.xml | 2 +- nifi/nifi-external/nifi-spark-receiver/pom.xml | 1 - .../nifi-aws-bundle/nifi-aws-processors/pom.xml | 2 +- .../nifi-framework-cluster/pom.xml | 2 +- .../nifi-framework/nifi-framework-core/pom.xml | 2 +- .../nifi-web/nifi-web-content-viewer/pom.xml| 2 +- .../nifi-framework/nifi-web/nifi-web-ui/pom.xml | 2 +- .../nifi-hl7-bundle/nifi-hl7-processors/pom.xml | 2 +- .../nifi-solr-processors/pom.xml| 2 +- .../nifi-standard-processors/pom.xml| 2 +- nifi/pom.xml| 45 +++- 12 files changed, 35 insertions(+), 31 deletions(-) --
Git Push Summary
Repository: incubator-nifi Updated Tags: refs/tags/nifi-nar-maven-plugin-1.0.1-incubating-RC11 [deleted] 64f4ced91
Git Push Summary
Repository: incubator-nifi Updated Tags: refs/tags/nifi-0.1.0-incubating-rc12 [deleted] a8f8ef6fb
Git Push Summary
Repository: incubator-nifi Updated Tags: refs/tags/nifi-nar-maven-plugin-1.0.1-incubating-rc12 [deleted] f4fa66cb5 refs/tags/nifi-parent-1.0.0-incubating-rc12 [deleted] 0a19f9fd0
[5/9] incubator-nifi git commit: JDBC driver can be loaded from external location using URLClassLoader
JDBC driver can be loaded from external location using URLClassLoader Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/1682d62d Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1682d62d Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1682d62d Branch: refs/heads/develop Commit: 1682d62d7f46ba854f77923352a77ff9f6a1b746 Parents: 864e099 Author: Toivo Adams toivo.ad...@gmail.com Authored: Sat Mar 14 12:34:40 2015 +0200 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 08:35:55 2015 -0400 -- .../apache/nifi/dbcp/DBCPConnectionPool.java| 23 -- .../java/org/apache/nifi/dbcp/DriverShim.java | 74 .../org/apache/nifi/dbcp/DBCPServiceTest.java | 35 + 3 files changed, 113 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1682d62d/nifi/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/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java index c2511c3..e2c0fea 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java @@ -20,6 +20,8 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; import java.sql.Connection; +import java.sql.Driver; +import java.sql.DriverManager; import java.sql.SQLException; import java.util.ArrayList; import java.util.Collections; @@ -184,7 +186,7 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC // Optional driver URL, when exist, this URL will be used to locate driver jar file location String urlString = context.getProperty(DB_DRIVER_JAR_URL).getValue(); -dataSource.setDriverClassLoader( getDriverClassLoader(urlString) ); +dataSource.setDriverClassLoader( getDriverClassLoader(urlString, drv) ); String dburl = dbsystem.buildUrl(host, port, dbname); @@ -210,14 +212,27 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC * using Thread.currentThread().getContextClassLoader(); * will ensure that you are using the ClassLoader for you NAR. * @throws InitializationException - */ -protected ClassLoader getDriverClassLoader(String urlString) throws InitializationException { + */ +protected ClassLoader getDriverClassLoader(String urlString, String drvName) throws InitializationException { if (urlString!=null urlString.length()0) { try { URL[] urls = new URL[] { new URL(urlString) }; - return new URLClassLoader(urls); + URLClassLoader ucl = new URLClassLoader(urls); + + // Workaround which allows to use URLClassLoader for JDBC driver loading. + // (Because the DriverManager will refuse to use a driver not loaded by the system ClassLoader.) + Class? clazz = Class.forName(drvName, true, ucl); + if (clazz==null) + throw new InitializationException(Can't load Database Driver + drvName); + Driver driver = (Driver) clazz.newInstance(); + DriverManager.registerDriver( new DriverShim(driver) ); + + return ucl; + } catch (MalformedURLException e) { throw new InitializationException(Invalid Database Driver Jar Url, e); + } catch (Exception e) { + throw new InitializationException(Can't load Database Driver, e); } } else
[4/9] incubator-nifi git commit: bump version to 0.1.0 , test cleanup
bump version to 0.1.0 , test cleanup Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/78906a94 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/78906a94 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/78906a94 Branch: refs/heads/develop Commit: 78906a9427757b5d387a18282191835089faebac Parents: 1682d62 Author: Toivo Adams toivo.ad...@gmail.com Authored: Sun Mar 22 15:10:37 2015 +0200 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 08:35:55 2015 -0400 -- .../nifi-dbcp-service-api/pom.xml | 2 +- .../nifi-dbcp-service-nar/pom.xml | 4 +- .../nifi-dbcp-service/pom.xml | 4 +- .../org/apache/nifi/dbcp/DBCPServiceTest.java | 58 +--- .../nifi-dbcp-service-bundle/pom.xml| 2 +- 5 files changed, 32 insertions(+), 38 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/78906a94/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml index f8ea08a..1eec223 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml @@ -5,7 +5,7 @@ parent groupIdorg.apache.nifi/groupId artifactIdnifi-standard-services/artifactId -version0.0.2-incubating-SNAPSHOT/version +version0.1.0-incubating-SNAPSHOT/version /parent artifactIdnifi-dbcp-service-api/artifactId packagingjar/packaging http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/78906a94/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service-nar/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service-nar/pom.xml index 37722db..e7cd28e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service-nar/pom.xml @@ -6,7 +6,7 @@ parent groupIdorg.apache.nifi/groupId artifactIdnifi-dbcp-service-bundle/artifactId -version0.0.2-incubating-SNAPSHOT/version +version0.1.0-incubating-SNAPSHOT/version /parent artifactIdnifi-dbcp-service-nar/artifactId packagingnar/packaging @@ -19,7 +19,7 @@ dependency groupIdorg.apache.nifi/groupId artifactIdnifi-dbcp-service/artifactId -version0.0.2-incubating-SNAPSHOT/version +version0.1.0-incubating-SNAPSHOT/version /dependency /dependencies http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/78906a94/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml index a17ddff..eec2a17 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml @@ -6,7 +6,7 @@ parent groupIdorg.apache.nifi/groupId artifactIdnifi-dbcp-service-bundle/artifactId -version0.0.2-incubating-SNAPSHOT/version +version0.1.0-incubating-SNAPSHOT/version /parent artifactIdnifi-dbcp-service/artifactId packagingjar/packaging @@ -14,7 +14,7 @@ dependency groupIdorg.apache.nifi/groupId artifactIdnifi-dbcp-service-api/artifactId -version0.0.2-incubating-SNAPSHOT/version +version0.1.0-incubating-SNAPSHOT/version /dependency dependency groupIdorg.apache.nifi/groupId http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/78906a94/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/DBCPServiceTest.java -- diff --git
[9/9] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/149ad130 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/149ad130 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/149ad130 Branch: refs/heads/develop Commit: 149ad130d91b06167db7fb45112d16fa341ad04b Parents: e0ee55e 8c7dd6a Author: Mark Payne marka...@hotmail.com Authored: Wed May 20 12:41:51 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 12:41:51 2015 -0400 -- nifi-site/Gruntfile.js| 10 ++ nifi-site/src/pages/html/download.hbs | 1 + 2 files changed, 11 insertions(+) --
[6/9] incubator-nifi git commit: NIFI-322: Fixed checkstyle errors, licensing issues, unit tests
NIFI-322: Fixed checkstyle errors, licensing issues, unit tests Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/91aaabfa Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/91aaabfa Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/91aaabfa Branch: refs/heads/develop Commit: 91aaabfa4c0533641568d82f0940402a28e5e8b7 Parents: 78906a9 Author: Mark Payne marka...@hotmail.com Authored: Wed May 20 09:53:36 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 09:53:36 2015 -0400 -- nifi/nifi-assembly/LICENSE | 29 ++ nifi/nifi-assembly/pom.xml | 6 + .../nifi-dbcp-service-api/pom.xml | 19 +- .../java/org/apache/nifi/dbcp/DBCPService.java | 6 +- .../nifi-dbcp-service-nar/pom.xml | 25 +- .../src/main/resources/META-INF/NOTICE | 22 ++ .../nifi-dbcp-service/pom.xml | 23 +- .../apache/nifi/dbcp/DBCPConnectionPool.java| 178 ++-- .../nifi/dbcp/DatabaseSystemDescriptor.java | 43 ++- .../org/apache/nifi/dbcp/DatabaseSystems.java | 106 +++ .../java/org/apache/nifi/dbcp/DriverShim.java | 84 +++--- .../org/apache/nifi/dbcp/DBCPServiceTest.java | 289 ++- .../apache/nifi/dbcp/TestDatabaseSystems.java | 18 +- .../org/apache/nifi/dbcp/TestProcessor.java | 12 +- .../nifi-dbcp-service-bundle/pom.xml| 21 +- .../nifi-standard-services-api-nar/pom.xml | 2 +- .../nifi-standard-services/pom.xml | 1 + nifi/pom.xml| 5 + 18 files changed, 497 insertions(+), 392 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/91aaabfa/nifi/nifi-assembly/LICENSE -- diff --git a/nifi/nifi-assembly/LICENSE b/nifi/nifi-assembly/LICENSE index ec360de..984bcea 100644 --- a/nifi/nifi-assembly/LICENSE +++ b/nifi/nifi-assembly/LICENSE @@ -851,3 +851,32 @@ For details see http://asm.ow2.org/asmdex-license.html ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +The binary distribution of this product bundles 'Hamcrest' which is available +under a BSD license. More details found here: http://hamcrest.org. + + Copyright (c) 2000-2006, www.hamcrest.org + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + Redistributions of source code must retain the above copyright notice, this list of + conditions and the following disclaimer. Redistributions in binary form must reproduce + the above copyright notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + Neither the name of Hamcrest nor the names of its contributors may be used to endorse + or promote products derived from this software without specific prior written + permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS AS IS AND ANY + EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES + OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT + SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED + TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR + BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY + WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH + DAMAGE. + \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/91aaabfa/nifi/nifi-assembly/pom.xml -- diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index cfe1de6..8c060aa 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -164,6 +164,12 @@ language governing permissions and limitations under the License. -- /dependency dependency groupIdorg.apache.nifi/groupId +artifactIdnifi-dbcp-service-nar/artifactId + version0.1.0-incubating-SNAPSHOT/version +typenar/type +/dependency +dependency +
[3/9] incubator-nifi git commit: DBCPConnectionPool service fixes
DBCPConnectionPool service fixes Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/d1cace6a Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d1cace6a Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d1cace6a Branch: refs/heads/develop Commit: d1cace6a8a3105d852a595a0e8ca0ec2d397293a Parents: 589e2b7 Author: Toivo Adams toivo.ad...@gmail.com Authored: Tue Mar 3 21:18:01 2015 +0200 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 08:35:54 2015 -0400 -- .../nifi-dbcp-service/pom.xml | 7 +- .../apache/nifi/dbcp/DBCPConnectionPool.java| 222 +++ .../nifi/dbcp/DBCPServiceApacheDBCP14.java | 208 - ...org.apache.nifi.controller.ControllerService | 15 ++ .../org/apache/nifi/dbcp/DBCPServiceTest.java | 54 ++--- 5 files changed, 270 insertions(+), 236 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1cace6a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml index d06a14c..a17ddff 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/pom.xml @@ -43,7 +43,12 @@ artifactIdderby/artifactId version10.11.1.1/version /dependency - + dependency + groupIdorg.hamcrest/groupId + artifactIdhamcrest-all/artifactId + version1.3/version + /dependency + /dependencies /project \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1cace6a/nifi/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/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java new file mode 100644 index 000..a80e9bf --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java @@ -0,0 +1,222 @@ +/* + * 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.dbcp; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.commons.dbcp.BasicDataSource; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnDisabled; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.reporting.InitializationException; + +/** + * Implementation of for Database Connection Pooling Service. + * Apache DBCP is used for connection pooling functionality. + * + */ +@Tags({dbcp, jdbc, database, connection, pooling,
[7/9] incubator-nifi git commit: Merge branch 'NIFI-322-Db-Connection-Pooling' into develop
Merge branch 'NIFI-322-Db-Connection-Pooling' into develop Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/281ebafc Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/281ebafc Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/281ebafc Branch: refs/heads/develop Commit: 281ebafc2c84c37c0bf0b27f5d17c732ab3662b4 Parents: a66848b 91aaabf Author: Mark Payne marka...@hotmail.com Authored: Wed May 20 09:54:16 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 09:54:16 2015 -0400 -- nifi/nifi-assembly/LICENSE | 29 ++ nifi/nifi-assembly/pom.xml | 6 + .../nifi-dbcp-service-api/pom.xml | 31 ++ .../java/org/apache/nifi/dbcp/DBCPService.java | 34 ++ .../nifi-dbcp-service-nar/pom.xml | 37 +++ .../src/main/resources/META-INF/NOTICE | 22 ++ .../nifi-dbcp-service/pom.xml | 65 .../apache/nifi/dbcp/DBCPConnectionPool.java| 269 +++ .../nifi/dbcp/DatabaseSystemDescriptor.java | 50 +++ .../org/apache/nifi/dbcp/DatabaseSystems.java | 82 + .../java/org/apache/nifi/dbcp/DriverShim.java | 74 + ...org.apache.nifi.controller.ControllerService | 15 + .../org/apache/nifi/dbcp/DBCPServiceTest.java | 331 +++ .../apache/nifi/dbcp/TestDatabaseSystems.java | 31 ++ .../org/apache/nifi/dbcp/TestProcessor.java | 45 +++ .../nifi-dbcp-service-bundle/pom.xml| 29 ++ .../nifi-standard-services-api-nar/pom.xml | 6 + .../nifi-standard-services/pom.xml | 2 + nifi/pom.xml| 5 + 19 files changed, 1163 insertions(+) --
[1/9] incubator-nifi git commit: Initial Database connection pooling service implementation NIFI-322
Repository: incubator-nifi Updated Branches: refs/heads/develop 8c7dd6ab3 - 149ad130d Initial Database connection pooling service implementation NIFI-322 Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/589e2b7e Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/589e2b7e Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/589e2b7e Branch: refs/heads/develop Commit: 589e2b7ebf6ef70307ff1a2ab67c473d2f19a9a9 Parents: 5273a63 Author: Toivo Adams toivo.ad...@gmail.com Authored: Sun Mar 1 22:27:06 2015 +0200 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 08:35:53 2015 -0400 -- .../nifi-dbcp-service-api/pom.xml | 18 ++ .../java/org/apache/nifi/dbcp/DBCPService.java | 38 .../nifi-dbcp-service-nar/pom.xml | 28 +++ .../nifi-dbcp-service/pom.xml | 49 + .../nifi/dbcp/DBCPServiceApacheDBCP14.java | 208 ++ .../nifi/dbcp/DatabaseSystemDescriptor.java | 51 + .../org/apache/nifi/dbcp/DatabaseSystems.java | 78 +++ .../org/apache/nifi/dbcp/DBCPServiceTest.java | 214 +++ .../apache/nifi/dbcp/TestDatabaseSystems.java | 37 .../org/apache/nifi/dbcp/TestProcessor.java | 47 .../nifi-dbcp-service-bundle/pom.xml| 18 ++ .../nifi-standard-services-api-nar/pom.xml | 6 + .../nifi-standard-services/pom.xml | 1 + 13 files changed, 793 insertions(+) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/589e2b7e/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml new file mode 100644 index 000..f8ea08a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/pom.xml @@ -0,0 +1,18 @@ +project xmlns=http://maven.apache.org/POM/4.0.0; xmlns:xsi=http://www.w3.org/2001/XMLSchema-instance; xsi:schemaLocation=http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd; + + +modelVersion4.0.0/modelVersion +parent +groupIdorg.apache.nifi/groupId +artifactIdnifi-standard-services/artifactId +version0.0.2-incubating-SNAPSHOT/version +/parent +artifactIdnifi-dbcp-service-api/artifactId +packagingjar/packaging +dependencies +dependency +groupIdorg.apache.nifi/groupId +artifactIdnifi-api/artifactId +/dependency +/dependencies +/project \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/589e2b7e/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/src/main/java/org/apache/nifi/dbcp/DBCPService.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/src/main/java/org/apache/nifi/dbcp/DBCPService.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/src/main/java/org/apache/nifi/dbcp/DBCPService.java new file mode 100644 index 000..8f2aa5a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-api/src/main/java/org/apache/nifi/dbcp/DBCPService.java @@ -0,0 +1,38 @@ +/* + * 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.dbcp; + +import java.sql.Connection; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.processor.exception.ProcessException; + +/** + * Definition for Database Connection Pooling Service. + * + */ +@Tags({dbcp, jdbc, database, connection, pooling,
[8/9] incubator-nifi git commit: NIFI-322: Fixed version number
NIFI-322: Fixed version number Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/e0ee55e6 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/e0ee55e6 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/e0ee55e6 Branch: refs/heads/develop Commit: e0ee55e6c44fc563b7a376857d0f81c8133bae74 Parents: 281ebaf Author: Mark Payne marka...@hotmail.com Authored: Wed May 20 10:53:22 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 10:53:22 2015 -0400 -- nifi/nifi-assembly/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e0ee55e6/nifi/nifi-assembly/pom.xml -- diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index 8c060aa..ba8dc24 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -165,7 +165,7 @@ language governing permissions and limitations under the License. -- dependency groupIdorg.apache.nifi/groupId artifactIdnifi-dbcp-service-nar/artifactId - version0.1.0-incubating-SNAPSHOT/version + version0.1.1-incubating-SNAPSHOT/version typenar/type /dependency dependency
[2/9] incubator-nifi git commit: Loading external JDBC driver using URLClassLoader
Loading external JDBC driver using URLClassLoader Signed-off-by: Toivo Adams toivo.ad...@gmail.com Signed-off-by: Mark Payne marka...@hotmail.com Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/864e0996 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/864e0996 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/864e0996 Branch: refs/heads/develop Commit: 864e0996ca0cbbe922d4db25ede0d3ad1e38c3c5 Parents: d1cace6 Author: Toivo Adams toivo.ad...@gmail.com Authored: Sun Mar 8 19:45:32 2015 +0200 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 08:35:54 2015 -0400 -- .../apache/nifi/dbcp/DBCPConnectionPool.java| 66 +++--- .../org/apache/nifi/dbcp/DBCPServiceTest.java | 125 +-- 2 files changed, 167 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/864e0996/nifi/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/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java index a80e9bf..c2511c3 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java @@ -16,11 +16,15 @@ */ package org.apache.nifi.dbcp; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLClassLoader; import java.sql.Connection; import java.sql.SQLException; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.commons.dbcp.BasicDataSource; import org.apache.nifi.annotation.documentation.CapabilityDescription; @@ -78,6 +82,14 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .build(); +public static final PropertyDescriptor DB_DRIVER_JAR_URL = new PropertyDescriptor.Builder() +.name(Database Driver Jar Url) +.description(Optional database driver jar file path url. For example 'file:///var/tmp/mariadb-java-client-1.1.7.jar') +.defaultValue(null) +.required(false) +.addValidator(StandardValidators.URL_VALIDATOR) +.build(); + public static final PropertyDescriptor DB_NAME = new PropertyDescriptor.Builder() .name(Database Name) .description(Database name) @@ -103,14 +115,14 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC .sensitive(true) .build(); -public static final PropertyDescriptor MAX_WAIT_MILLIS = new PropertyDescriptor.Builder() -.name(Max Wait Millis) -.description(The maximum number of milliseconds that the pool will wait (when there are no available connections) - + for a connection to be returned before throwing an exception, or -1 to wait indefinitely. ) -.defaultValue(500) +public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder() +.name(Max Wait Time) +.description(The maximum amount of time that the pool will wait (when there are no available connections) + + for a connection to be returned before failing, or -1 to wait indefinitely. ) +.defaultValue(500 millis) .required(true) -.addValidator(StandardValidators.LONG_VALIDATOR) -.sensitive(true) +.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) +.sensitive(false) .build(); public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder() @@ -131,14 +143,16 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC props.add(DB_HOST); props.add(DB_PORT); props.add(DB_DRIVERNAME); +props.add(DB_DRIVER_JAR_URL); props.add(DB_NAME); props.add(DB_USER); props.add(DB_PASSWORD); +props.add(MAX_WAIT_TIME); +props.add(MAX_TOTAL_CONNECTIONS); properties = Collections.unmodifiableList(props); } -private ConfigurationContext configContext; private volatile BasicDataSource dataSource; @Override @@ -153,7 +167,6 @@ public class DBCPConnectionPool extends AbstractControllerService
[1/3] incubator-nifi git commit: NIFI-446 Adding a Delimiter Strategy to MergeContent
Repository: incubator-nifi Updated Branches: refs/heads/develop 149ad130d - 82174e460 NIFI-446 Adding a Delimiter Strategy to MergeContent Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/3cf65261 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/3cf65261 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/3cf65261 Branch: refs/heads/develop Commit: 3cf65261aa141ef116b762d821cc5dbe26bb2bdd Parents: 5273a63 Author: bbende bbe...@gmail.com Authored: Tue May 19 21:11:50 2015 -0400 Committer: bbende bbe...@gmail.com Committed: Tue May 19 21:12:24 2015 -0400 -- .../nifi/processors/standard/MergeContent.java | 92 +-- .../processors/standard/TestMergeContent.java | 97 2 files changed, 179 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3cf65261/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java -- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index b11dee3..2883a75 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -23,6 +23,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -49,6 +50,8 @@ import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.ProcessContext; @@ -130,6 +133,11 @@ public class MergeContent extends BinFiles { + have the attributes fragment.identifier, fragment.count, and fragment.index or alternatively (for backward compatibility + purposes) segment.identifier, segment.count, and segment.index); +public static final AllowableValue DELIMITER_STRATEGY_FILENAME = new AllowableValue( +Filename, Filename, The values of Header, Footer, and Demarcator will be retrieved from the contents of a file); +public static final AllowableValue DELIMITER_STRATEGY_TEXT = new AllowableValue( +Text, Text, The values of Header, Footer, and Demarcator will be specified as property values); + public static final String MERGE_FORMAT_TAR_VALUE = TAR; public static final String MERGE_FORMAT_ZIP_VALUE = ZIP; public static final String MERGE_FORMAT_FLOWFILE_STREAM_V3_VALUE = FlowFile Stream, v3; @@ -210,26 +218,40 @@ public class MergeContent extends BinFiles { .defaultValue(null) .build(); +public static final PropertyDescriptor DELIMITER_STRATEGY = new PropertyDescriptor.Builder() +.required(true) +.name(Delimiter Strategy) +.description(Determines if Header, Footer, and Demarcator should point to files containing the respective content, or if ++ the values of the properties should be used as the content.) +.allowableValues(DELIMITER_STRATEGY_FILENAME, DELIMITER_STRATEGY_TEXT) +.defaultValue(DELIMITER_STRATEGY_FILENAME.getValue()) +.build(); public static final PropertyDescriptor HEADER = new PropertyDescriptor.Builder() .name(Header File) +.displayName(Header) .description(Filename specifying the header to use. If not specified, no header is supplied. This property is valid only when using the + binary-concatenation merge strategy; otherwise, it is ignored.) .required(false) -.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) +.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.expressionLanguageSupported(true) .build(); public static final PropertyDescriptor FOOTER = new PropertyDescriptor.Builder()
[2/3] incubator-nifi git commit: Merge branch 'NIFI-446' of https://github.com/bbende/incubator-nifi into NIFI-446-merge-content-delimiter-strategy
Merge branch 'NIFI-446' of https://github.com/bbende/incubator-nifi into NIFI-446-merge-content-delimiter-strategy Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f247e515 Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f247e515 Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f247e515 Branch: refs/heads/develop Commit: f247e515dcd8f6f2ec1ddeb95dd4dfa6a0dfb165 Parents: 149ad13 3cf6526 Author: Mark Payne marka...@hotmail.com Authored: Wed May 20 14:47:53 2015 -0400 Committer: Mark Payne marka...@hotmail.com Committed: Wed May 20 14:47:53 2015 -0400 -- .../nifi/processors/standard/MergeContent.java | 92 +-- .../processors/standard/TestMergeContent.java | 97 2 files changed, 179 insertions(+), 10 deletions(-) --