From df223a71afd34b75ca2d0e8cfaabc045de6c09dd Mon Sep 17 00:00:00 2001 From: Mathias Tiberghien Date: Wed, 5 Oct 2016 17:50:08 +0200 Subject: [PATCH 1/6] 1 first commit for feature NIFI-1526 DefaultSchedule annotation should be use on Custom Processor to set the default scheduling strategy, scheduling period or max number of concurrent task for each instance of the processor DefaultSettings annotation should be use on Custom Processor to set the default penalty period, the yield duration or the bulletin log level for each instance of the processor --- .../configuration/DefaultSchedule.java | 23 ++++++++++ .../configuration/DefaultSettings.java | 21 +++++++++ .../nifi/controller/FlowController.java | 37 +++++++++++++++ .../controller/StandardProcessorNode.java | 46 ++++++++++++++++++- 4 files changed, 126 insertions(+), 1 deletion(-) create mode 100644 nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java create mode 100644 nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java new file mode 100644 index 000000000000..a5b90bc4b9a3 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java @@ -0,0 +1,23 @@ +package org.apache.nifi.annotation.configuration; + +import org.apache.nifi.scheduling.SchedulingStrategy; + +import java.lang.annotation.*; + +/** + *

+ * Marker interface that a Processor can use to configure the schedule strategy, the period and the number of concurrent tasks. + * Note that the number of Concurrent tasks will be ignored if the annotion @TriggerSerialy is used + *

+ */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface DefaultSchedule { + + SchedulingStrategy Strategy() default SchedulingStrategy.TIMER_DRIVEN; + String Period() default "0 sec"; + int ConcurrentTasks() default 1; + +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java new file mode 100644 index 000000000000..db1263d73933 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java @@ -0,0 +1,21 @@ +package org.apache.nifi.annotation.configuration; + +import org.apache.nifi.logging.LogLevel; + +import java.lang.annotation.*; + +/** + *

+ * Marker interface that a Processor can use to configure the yield duration, the penalty duration and the bulletin log level. + * Note that the number of Concurrent tasks will be ignored if the annotion @TriggerSerialy is used + *

+ */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface DefaultSettings { + String YieldDuration() default "1 sec"; + String PenaltyDuration() default "30 sec"; + LogLevel LogLevel() default LogLevel.WARN; +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index b42f3ae550c4..ec3049a41e97 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -49,6 +49,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.nifi.action.Action; import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.annotation.configuration.DefaultSettings; import org.apache.nifi.annotation.lifecycle.OnAdded; import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.annotation.lifecycle.OnRemoved; @@ -1059,6 +1060,42 @@ public ProcessorNode createProcessor(final String type, String id, final boolean final LogRepository logRepository = LogRepositoryFactory.getRepository(id); logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ProcessorLogObserver(getBulletinRepository(), procNode)); + try { + + final Class procClass = processor.getClass(); + if(procClass.isAnnotationPresent(DefaultSettings.class)) + { + DefaultSettings ds = procClass.getAnnotation(DefaultSettings.class); + try { + procNode.setYieldPeriod(ds.YieldDuration()); + } + catch(Throwable ex) + { + LOG.error(String.format("Error while setting yield period from DefaultSettings annotation:%s",ex.getMessage())); + } + try { + + procNode.setPenalizationPeriod(ds.PenaltyDuration()); + } + catch(Throwable ex) + { + LOG.error(String.format("Error while setting penalty duration from DefaultSettings annotation:%s",ex.getMessage())); + } + try { + procNode.setBulletinLevel(ds.LogLevel()); + } + catch (Throwable ex) + { + LOG.error(String.format("Error while setting bulletin level from DefaultSettings annotation:%s",ex.getMessage())); + } + + } + } + catch (Throwable ex) + { + LOG.error(String.format("Error while setting default settings from DefaultSettings annotation: %s",ex.getMessage())); + } + if (firstTimeAdded) { try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getClass())) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index 42790fdca98e..4ab4c33c0781 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -26,6 +26,7 @@ import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.configuration.DefaultSchedule; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; @@ -148,7 +149,6 @@ public StandardProcessorNode(final Processor processor, final String uuid, final String componentType, final String componentCanonicalClass, final NiFiProperties nifiProperties) { super(processor, uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass); - this.processor = processor; identifier = new AtomicReference<>(uuid); destinations = new HashMap<>(); @@ -187,6 +187,50 @@ public StandardProcessorNode(final Processor processor, final String uuid, } schedulingStrategy = SchedulingStrategy.TIMER_DRIVEN; + try + { + + if(procClass.isAnnotationPresent(DefaultSchedule.class)) + { + DefaultSchedule dsc = procClass.getAnnotation(DefaultSchedule.class); + try + { + this.setSchedulingStrategy(dsc.Strategy()); + } + catch (Throwable ex) + { + LOG.error(String.format("Error while setting scheduling strategy from DefaultSchedule annotation: %s",ex.getMessage())); + } + try + { + this.setScheduldingPeriod(dsc.Period()); + } + catch (Throwable ex) + { + this.setSchedulingStrategy(SchedulingStrategy.TIMER_DRIVEN); + LOG.error(String.format("Error while setting scheduling period from DefaultSchedule annotation: %s",ex.getMessage())); + } + if(!triggeredSerially) + { + try + { + setMaxConcurrentTasks(dsc.ConcurrentTasks()); + } + catch (Throwable ex) + { + LOG.error(String.format("Error while setting max concurrent tasks from DefaultSchedule annotation: %s",ex.getMessage())); + } + + } + } + } + catch (Throwable ex) + { + LOG.error(String.format("Error while setting default schedule from DefaultSchedule annotation: %s",ex.getMessage())); + } + + + } /** From 65bb92d0c438b5b330bdc500a7a8ebdf066653ce Mon Sep 17 00:00:00 2001 From: Mathias Tiberghien Date: Fri, 7 Oct 2016 15:07:44 +0200 Subject: [PATCH 2/6] Corrected Code Styles Added Exception to log to have the StackTrace Added Dummy_ScheduledProcessor and Dummy_SettingsProcessor to nifi-framework-core\src\test\java\org_apache\nifi\controller to test annotations Added tests methods to nifi-framework-core\src\test\java\org\apache\nifi\controller\TestFlowController.java : the methods create a Processor using the FlowController and check that the properties are those defined on annotations of the dummy processor classes --- .../configuration/DefaultSchedule.java | 7 ++- .../configuration/DefaultSettings.java | 8 +++- .../nifi/controller/FlowController.java | 25 ++++------- .../controller/StandardProcessorNode.java | 44 ++++++------------- .../controller/Dummy_ScheduledProcessor.java | 20 +++++++++ .../controller/Dummy_SettingsProcessor.java | 19 ++++++++ .../nifi/controller/TestFlowController.java | 34 +++++++++++++- 7 files changed, 105 insertions(+), 52 deletions(-) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_ScheduledProcessor.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_SettingsProcessor.java diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java index a5b90bc4b9a3..c4f1ac80885d 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java @@ -2,7 +2,12 @@ import org.apache.nifi.scheduling.SchedulingStrategy; -import java.lang.annotation.*; +import java.lang.annotation.Documented; +import java.lang.annotation.Target; +import java.lang.annotation.Retention; +import java.lang.annotation.ElementType; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Inherited; /** *

diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java index db1263d73933..4732f6e4d694 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java @@ -1,9 +1,13 @@ package org.apache.nifi.annotation.configuration; +import java.lang.annotation.Documented; +import java.lang.annotation.Target; +import java.lang.annotation.Retention; +import java.lang.annotation.ElementType; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Inherited; import org.apache.nifi.logging.LogLevel; -import java.lang.annotation.*; - /** *

* Marker interface that a Processor can use to configure the yield duration, the penalty duration and the bulletin log level. diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index ec3049a41e97..363d58dfae13 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -1061,39 +1061,30 @@ public ProcessorNode createProcessor(final String type, String id, final boolean logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ProcessorLogObserver(getBulletinRepository(), procNode)); try { - final Class procClass = processor.getClass(); if(procClass.isAnnotationPresent(DefaultSettings.class)) { DefaultSettings ds = procClass.getAnnotation(DefaultSettings.class); try { procNode.setYieldPeriod(ds.YieldDuration()); - } - catch(Throwable ex) - { - LOG.error(String.format("Error while setting yield period from DefaultSettings annotation:%s",ex.getMessage())); + } catch(Throwable ex) { + LOG.error(String.format("Error while setting yield period from DefaultSettings annotation:%s",ex.getMessage()),ex); } try { procNode.setPenalizationPeriod(ds.PenaltyDuration()); - } - catch(Throwable ex) - { - LOG.error(String.format("Error while setting penalty duration from DefaultSettings annotation:%s",ex.getMessage())); + } catch(Throwable ex) { + LOG.error(String.format("Error while setting penalty duration from DefaultSettings annotation:%s",ex.getMessage()),ex); } try { procNode.setBulletinLevel(ds.LogLevel()); - } - catch (Throwable ex) - { - LOG.error(String.format("Error while setting bulletin level from DefaultSettings annotation:%s",ex.getMessage())); + } catch (Throwable ex) { + LOG.error(String.format("Error while setting bulletin level from DefaultSettings annotation:%s",ex.getMessage()),ex); } } - } - catch (Throwable ex) - { - LOG.error(String.format("Error while setting default settings from DefaultSettings annotation: %s",ex.getMessage())); + } catch (Throwable ex) { + LOG.error(String.format("Error while setting default settings from DefaultSettings annotation: %s",ex.getMessage()),ex); } if (firstTimeAdded) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index 4ab4c33c0781..cfb08b8765ae 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -187,46 +187,30 @@ public StandardProcessorNode(final Processor processor, final String uuid, } schedulingStrategy = SchedulingStrategy.TIMER_DRIVEN; - try - { - - if(procClass.isAnnotationPresent(DefaultSchedule.class)) - { + try { + if (procClass.isAnnotationPresent(DefaultSchedule.class)) { DefaultSchedule dsc = procClass.getAnnotation(DefaultSchedule.class); - try - { + try { this.setSchedulingStrategy(dsc.Strategy()); + } catch (Throwable ex) { + LOG.error(String.format("Error while setting scheduling strategy from DefaultSchedule annotation: %s", ex.getMessage()), ex); } - catch (Throwable ex) - { - LOG.error(String.format("Error while setting scheduling strategy from DefaultSchedule annotation: %s",ex.getMessage())); - } - try - { + try { this.setScheduldingPeriod(dsc.Period()); - } - catch (Throwable ex) - { + } catch (Throwable ex) { this.setSchedulingStrategy(SchedulingStrategy.TIMER_DRIVEN); - LOG.error(String.format("Error while setting scheduling period from DefaultSchedule annotation: %s",ex.getMessage())); + LOG.error(String.format("Error while setting scheduling period from DefaultSchedule annotation: %s", ex.getMessage()), ex); } - if(!triggeredSerially) - { - try - { + if (!triggeredSerially) { + try { setMaxConcurrentTasks(dsc.ConcurrentTasks()); + } catch (Throwable ex) { + LOG.error(String.format("Error while setting max concurrent tasks from DefaultSchedule annotation: %s", ex.getMessage()), ex); } - catch (Throwable ex) - { - LOG.error(String.format("Error while setting max concurrent tasks from DefaultSchedule annotation: %s",ex.getMessage())); - } - } } - } - catch (Throwable ex) - { - LOG.error(String.format("Error while setting default schedule from DefaultSchedule annotation: %s",ex.getMessage())); + } catch (Throwable ex) { + LOG.error(String.format("Error while setting default schedule from DefaultSchedule annotation: %s",ex.getMessage()),ex); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_ScheduledProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_ScheduledProcessor.java new file mode 100644 index 000000000000..a9cc4abbdad4 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_ScheduledProcessor.java @@ -0,0 +1,20 @@ +package org.apache.nifi.controller; + +import org.apache.nifi.annotation.configuration.DefaultSchedule; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.scheduling.SchedulingStrategy; + + +/** + * Dummy processor to test @DefaultSchedule annotation + */ +@DefaultSchedule(ConcurrentTasks = 5, Strategy = SchedulingStrategy.CRON_DRIVEN,Period = "0 0 0 1/1 * ?") +public class Dummy_ScheduledProcessor extends AbstractProcessor { + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_SettingsProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_SettingsProcessor.java new file mode 100644 index 000000000000..00b980e9d1be --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_SettingsProcessor.java @@ -0,0 +1,19 @@ +package org.apache.nifi.controller; + +import org.apache.nifi.annotation.configuration.DefaultSettings; +import org.apache.nifi.logging.LogLevel; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; + +/** + * Dummy Processor to test @DefaultSettings annotation + */ +@DefaultSettings(YieldDuration = "5 sec", PenaltyDuration = "1 min", LogLevel = LogLevel.DEBUG) +public class Dummy_SettingsProcessor extends AbstractProcessor { + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java index bbcdc3b7b8af..b25e435fca2a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java @@ -46,10 +46,12 @@ import org.apache.nifi.controller.repository.FlowFileEventRepository; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.encrypt.StringEncryptor; -import org.apache.nifi.processor.Relationship; +import org.apache.nifi.logging.LogLevel; +import org.apache.nifi.processor.*; import org.apache.nifi.provenance.MockProvenanceRepository; import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.FileBasedVariableRegistry; import org.apache.nifi.util.NiFiProperties; import org.junit.After; @@ -57,6 +59,7 @@ import org.junit.Test; import org.mockito.Mockito; + public class TestFlowController { private FlowController controller; @@ -71,7 +74,7 @@ public class TestFlowController { @Before public void setup() { - System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFlowController.class.getResource("/nifi.properties").getFile()); + System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, FlowController.class.getResource("/nifi.properties").getFile()); flowFileEventRepo = Mockito.mock(FlowFileEventRepository.class); auditService = Mockito.mock(AuditService.class); @@ -327,4 +330,31 @@ public void testCreateMissingControllerService() throws ProcessorInstantiationEx assertFalse(service.equals(serviceNode)); } + @Test + public void testProcessorDefaultScheduleAnnotation() throws ProcessorInstantiationException,ClassNotFoundException,InstantiationException,IllegalAccessException { + ProcessorNode p_scheduled = controller.createProcessor(Dummy_ScheduledProcessor.class.getName(),"1234-ScheduledProcessor"); + assertEquals(5,p_scheduled.getMaxConcurrentTasks()); + assertEquals(SchedulingStrategy.CRON_DRIVEN,p_scheduled.getSchedulingStrategy()); + assertEquals("0 0 0 1/1 * ?",p_scheduled.getSchedulingPeriod()); + assertEquals("1 sec",p_scheduled.getYieldPeriod()); + assertEquals("30 sec",p_scheduled.getPenalizationPeriod()); + assertEquals(LogLevel.WARN,p_scheduled.getBulletinLevel()); + } + + @Test + public void testProcessorDefaultSettingsAnnotation() throws ProcessorInstantiationException,ClassNotFoundException { + + ProcessorNode p_settings = controller.createProcessor(Dummy_SettingsProcessor.class.getName(),"1234-SettingsProcessor"); + assertEquals("5 sec",p_settings.getYieldPeriod()); + assertEquals("1 min",p_settings.getPenalizationPeriod()); + assertEquals(LogLevel.DEBUG,p_settings.getBulletinLevel()); + assertEquals(1,p_settings.getMaxConcurrentTasks()); + assertEquals(SchedulingStrategy.TIMER_DRIVEN,p_settings.getSchedulingStrategy()); + assertEquals("0 sec",p_settings.getSchedulingPeriod()); + } + + + + + } From 44a83997bdbbb101897f8cfb444efcc251e98094 Mon Sep 17 00:00:00 2001 From: Mathias Tiberghien Date: Sat, 8 Oct 2016 15:35:59 +0200 Subject: [PATCH 3/6] Removed '_' in Dummy classes lowercase methods in annotations Removed * in TestFlowController class import Replaced FlowController by TestFlowController in setup method of TestFlowController class --- .../annotation/behavior/TriggerWhenEmpty.java | 2 +- .../configuration/DefaultSchedule.java | 6 +++--- .../configuration/DefaultSettings.java | 6 +++--- .../processor/util/StandardValidators.java | 4 ++-- .../nifi/processors/avro/SplitAvro.java | 8 ++++---- .../apache/nifi/controller/AbstractPort.java | 2 +- .../nifi/controller/StandardFunnel.java | 2 +- .../nifi/controller/FlowController.java | 9 ++++----- .../controller/StandardFlowFileQueue.java | 2 +- .../controller/StandardProcessorNode.java | 12 +++++------ .../logging/ControllerServiceLogObserver.java | 4 ++-- .../nifi/logging/ProcessorLogObserver.java | 4 ++-- .../logging/ReportingTaskLogObserver.java | 4 ++-- .../remote/StandardRemoteProcessGroup.java | 6 +++--- ...ssor.java => DummyScheduledProcessor.java} | 4 ++-- ...essor.java => DummySettingsProcessor.java} | 4 ++-- .../nifi/controller/TestFlowController.java | 8 ++++---- .../apache/nifi/audit/ProcessorAuditor.java | 2 +- .../web/dao/impl/StandardProcessorDAO.java | 4 ++-- .../dao/impl/StandardReportingTaskDAO.java | 4 ++-- .../hadoop/AbstractHadoopProcessor.java | 4 ++-- .../nifi/processors/hadoop/PutHDFS.java | 2 +- .../apache/nifi/hbase/AbstractPutHBase.java | 2 +- .../org/apache/nifi/hbase/PutHBaseJSON.java | 6 +++--- .../nifi/processors/kafka/PutKafka.java | 4 ++-- .../org/apache/nifi/ldap/LdapProvider.java | 4 ++-- .../nifi/processors/splunk/GetSplunk.java | 8 ++++---- .../processors/standard/DistributeLoad.java | 10 +++++----- .../nifi/processors/standard/FetchFile.java | 20 +++++++++---------- .../standard/FetchFileTransfer.java | 6 +++--- .../processors/standard/MergeContent.java | 16 +++++++-------- .../nifi/processors/standard/PutFile.java | 2 +- .../nifi/processors/standard/ReplaceText.java | 8 ++++---- .../processors/standard/RouteOnAttribute.java | 4 ++-- .../nifi/processors/standard/RouteText.java | 10 +++++----- .../nifi/processors/standard/TailFile.java | 2 +- .../nifi/processors/standard/ValidateCsv.java | 2 +- .../apache/nifi/controller/MonitorMemory.java | 2 +- .../cache/server/DistributedCacheServer.java | 2 +- 39 files changed, 105 insertions(+), 106 deletions(-) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/{Dummy_ScheduledProcessor.java => DummyScheduledProcessor.java} (76%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/{Dummy_SettingsProcessor.java => DummySettingsProcessor.java} (77%) diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java index 6fac03ba752a..bb23f22c9e0b 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java @@ -31,7 +31,7 @@ *

* *

- * A Processor is scheduled to be triggered based on its configured Scheduling Period + * A Processor is scheduled to be triggered based on its configured Scheduling period * and Scheduling Strategy. However, when the scheduling period elapses, the Processor * will not be scheduled if it has no work to do. Normally, a Processor is said to have * work to do if one of the following circumstances is true: diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java index c4f1ac80885d..10e6677c36b4 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java @@ -21,8 +21,8 @@ @Inherited public @interface DefaultSchedule { - SchedulingStrategy Strategy() default SchedulingStrategy.TIMER_DRIVEN; - String Period() default "0 sec"; - int ConcurrentTasks() default 1; + SchedulingStrategy strategy() default SchedulingStrategy.TIMER_DRIVEN; + String period() default "0 sec"; + int concurrentTasks() default 1; } diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java index 4732f6e4d694..5800b053ef61 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java @@ -19,7 +19,7 @@ @Retention(RetentionPolicy.RUNTIME) @Inherited public @interface DefaultSettings { - String YieldDuration() default "1 sec"; - String PenaltyDuration() default "30 sec"; - LogLevel LogLevel() default LogLevel.WARN; + String yieldDuration() default "1 sec"; + String penaltyDuration() default "30 sec"; + LogLevel logLevel() default LogLevel.WARN; } diff --git a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index de1e57f59182..6eb3a8d7cb65 100644 --- a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -319,7 +319,7 @@ public ValidationResult validate(final String subject, final String input, final } if (input == null) { - return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time period cannot be null").build(); } if (Pattern.compile(FormatUtils.TIME_DURATION_REGEX).matcher(input.toLowerCase()).matches()) { return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); @@ -652,7 +652,7 @@ public ValidationResult validate(final String subject, final String input, final } if (input == null) { - return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time period cannot be null").build(); } final String lowerCase = input.toLowerCase(); final boolean validSyntax = pattern.matcher(lowerCase).matches(); diff --git a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java index e3eb6ec577f8..b9de350536cf 100644 --- a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java +++ b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java @@ -68,7 +68,7 @@ @SupportsBatching @Tags({ "avro", "split" }) @InputRequirement(Requirement.INPUT_REQUIRED) -@CapabilityDescription("Splits a binary encoded Avro datafile into smaller files based on the configured Output Size. The Output Strategy determines if " + +@CapabilityDescription("Splits a binary encoded Avro datafile into smaller files based on the configured Output Size. The Output strategy determines if " + "the smaller files will be Avro datafiles, or bare Avro records with metadata in the FlowFile attributes. The output will always be binary encoded.") @WritesAttributes({ @WritesAttribute(attribute = "fragment.identifier", @@ -85,7 +85,7 @@ public class SplitAvro extends AbstractProcessor { public static final AllowableValue RECORD_SPLIT = new AllowableValue(RECORD_SPLIT_VALUE, RECORD_SPLIT_VALUE, "Split at Record boundaries"); public static final PropertyDescriptor SPLIT_STRATEGY = new PropertyDescriptor.Builder() - .name("Split Strategy") + .name("Split strategy") .description("The strategy for splitting the incoming datafile. The Record strategy will read the incoming datafile by de-serializing each record.") .required(true) .allowableValues(RECORD_SPLIT) @@ -108,7 +108,7 @@ public class SplitAvro extends AbstractProcessor { public static final AllowableValue BARE_RECORD_OUTPUT = new AllowableValue(BARE_RECORD_OUTPUT_VALUE, BARE_RECORD_OUTPUT_VALUE, "Bare Avro records"); public static final PropertyDescriptor OUTPUT_STRATEGY = new PropertyDescriptor.Builder() - .name("Output Strategy") + .name("Output strategy") .description("Determines the format of the output. Either Avro Datafile, or bare record. Bare record output is only intended for use with systems " + "that already require it, and shouldn't be needed for normal use.") .required(true) @@ -118,7 +118,7 @@ public class SplitAvro extends AbstractProcessor { public static final PropertyDescriptor TRANSFER_METADATA = new PropertyDescriptor.Builder() .name("Transfer Metadata") - .description("Whether or not to transfer metadata from the parent datafile to the children. If the Output Strategy is Bare Record, " + + .description("Whether or not to transfer metadata from the parent datafile to the children. If the Output strategy is Bare Record, " + "then the metadata will be stored as FlowFile attributes, otherwise it will be in the Datafile header.") .required(true) .allowableValues("true", "false") diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java index 581d209982b7..eac5b914879f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java @@ -465,7 +465,7 @@ public void setYieldPeriod(final String yieldPeriod) { public void setScheduldingPeriod(final String schedulingPeriod) { final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); if (schedulingNanos < 0) { - throw new IllegalArgumentException("Scheduling Period must be positive"); + throw new IllegalArgumentException("Scheduling period must be positive"); } this.schedulingPeriod.set(schedulingPeriod); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java index c4cdf345f875..c56113ccfc75 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java @@ -425,7 +425,7 @@ public void setYieldPeriod(final String yieldPeriod) { public void setScheduldingPeriod(final String schedulingPeriod) { final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); if (schedulingNanos < 0) { - throw new IllegalArgumentException("Scheduling Period must be positive"); + throw new IllegalArgumentException("Scheduling period must be positive"); } this.schedulingPeriod.set(schedulingPeriod); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 363d58dfae13..41fa3d35d339 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -1062,22 +1062,21 @@ public ProcessorNode createProcessor(final String type, String id, final boolean try { final Class procClass = processor.getClass(); - if(procClass.isAnnotationPresent(DefaultSettings.class)) - { + if(procClass.isAnnotationPresent(DefaultSettings.class)) { DefaultSettings ds = procClass.getAnnotation(DefaultSettings.class); try { - procNode.setYieldPeriod(ds.YieldDuration()); + procNode.setYieldPeriod(ds.yieldDuration()); } catch(Throwable ex) { LOG.error(String.format("Error while setting yield period from DefaultSettings annotation:%s",ex.getMessage()),ex); } try { - procNode.setPenalizationPeriod(ds.PenaltyDuration()); + procNode.setPenalizationPeriod(ds.penaltyDuration()); } catch(Throwable ex) { LOG.error(String.format("Error while setting penalty duration from DefaultSettings annotation:%s",ex.getMessage()),ex); } try { - procNode.setBulletinLevel(ds.LogLevel()); + procNode.setBulletinLevel(ds.logLevel()); } catch (Throwable ex) { LOG.error(String.format("Error while setting bulletin level from DefaultSettings annotation:%s",ex.getMessage()),ex); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java index 68af208983ec..b79dee1b8fdc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java @@ -802,7 +802,7 @@ public int getFlowFileExpiration(final TimeUnit timeUnit) { public void setFlowFileExpiration(final String flowExpirationPeriod) { final long millis = FormatUtils.getTimeDuration(flowExpirationPeriod, TimeUnit.MILLISECONDS); if (millis < 0) { - throw new IllegalArgumentException("FlowFile Expiration Period must be positive"); + throw new IllegalArgumentException("FlowFile Expiration period must be positive"); } expirationPeriod.set(new TimePeriod(flowExpirationPeriod, millis)); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index cfb08b8765ae..d75a22868011 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -191,19 +191,19 @@ public StandardProcessorNode(final Processor processor, final String uuid, if (procClass.isAnnotationPresent(DefaultSchedule.class)) { DefaultSchedule dsc = procClass.getAnnotation(DefaultSchedule.class); try { - this.setSchedulingStrategy(dsc.Strategy()); + this.setSchedulingStrategy(dsc.strategy()); } catch (Throwable ex) { LOG.error(String.format("Error while setting scheduling strategy from DefaultSchedule annotation: %s", ex.getMessage()), ex); } try { - this.setScheduldingPeriod(dsc.Period()); + this.setScheduldingPeriod(dsc.period()); } catch (Throwable ex) { this.setSchedulingStrategy(SchedulingStrategy.TIMER_DRIVEN); LOG.error(String.format("Error while setting scheduling period from DefaultSchedule annotation: %s", ex.getMessage()), ex); } if (!triggeredSerially) { try { - setMaxConcurrentTasks(dsc.ConcurrentTasks()); + setMaxConcurrentTasks(dsc.concurrentTasks()); } catch (Throwable ex) { LOG.error(String.format("Error while setting max concurrent tasks from DefaultSchedule annotation: %s", ex.getMessage()), ex); } @@ -430,7 +430,7 @@ public boolean isEventDrivenSupported() { } /** - * Updates the Scheduling Strategy used for this Processor + * Updates the Scheduling strategy used for this Processor * * @param schedulingStrategy * strategy @@ -479,7 +479,7 @@ public void setScheduldingPeriod(final String schedulingPeriod) { new CronExpression(schedulingPeriod); } catch (final Exception e) { throw new IllegalArgumentException( - "Scheduling Period is not a valid cron expression: " + schedulingPeriod); + "Scheduling period is not a valid cron expression: " + schedulingPeriod); } } break; @@ -488,7 +488,7 @@ public void setScheduldingPeriod(final String schedulingPeriod) { final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); if (schedulingNanos < 0) { - throw new IllegalArgumentException("Scheduling Period must be positive"); + throw new IllegalArgumentException("Scheduling period must be positive"); } this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos)); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java index 3ea432d40107..de5b2aa55a4a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java @@ -35,8 +35,8 @@ public ControllerServiceLogObserver(final BulletinRepository bulletinRepository, @Override public void onLogMessage(final LogMessage message) { - // Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever - // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are). + // Map logLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever + // the logLevel is (INFO and ERROR map directly and all others we will just accept as they are). final String bulletinLevel = message.getLevel() == LogLevel.WARN ? Severity.WARNING.name() : message.getLevel().toString(); final Bulletin bulletin = BulletinFactory.createBulletin(null, serviceNode.getIdentifier(), ComponentType.CONTROLLER_SERVICE, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java index 62b90d63421c..a7eb03b7ff6d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java @@ -38,8 +38,8 @@ public ProcessorLogObserver(BulletinRepository bulletinRepository, ProcessorNode @Override public void onLogMessage(final LogMessage message) { - // Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever - // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are). + // Map logLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever + // the logLevel is (INFO and ERROR map directly and all others we will just accept as they are). final String bulletinLevel = (message.getLevel() == LogLevel.WARN) ? Severity.WARNING.name() : message.getLevel().toString(); bulletinRepository.addBulletin(BulletinFactory.createBulletin(processorNode, CATEGORY, bulletinLevel, message.getMessage())); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java index f52bc1ccd61d..235aeffb5c5b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java @@ -34,8 +34,8 @@ public ReportingTaskLogObserver(final BulletinRepository bulletinRepository, fin @Override public void onLogMessage(final LogMessage message) { - // Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever - // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are). + // Map logLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever + // the logLevel is (INFO and ERROR map directly and all others we will just accept as they are). final String bulletinLevel = message.getLevel() == LogLevel.WARN ? Severity.WARNING.name() : message.getLevel().toString(); final Bulletin bulletin = BulletinFactory.createBulletin(null, taskNode.getIdentifier(), ComponentType.REPORTING_TASK, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java index 31bfd3dfba10..c7dfe4756454 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java @@ -328,14 +328,14 @@ public void setCommunicationsTimeout(final String timePeriod) throws IllegalArgu try { final long millis = FormatUtils.getTimeDuration(timePeriod, TimeUnit.MILLISECONDS); if (millis <= 0) { - throw new IllegalArgumentException("Time Period must be more than 0 milliseconds; Invalid Time Period: " + timePeriod); + throw new IllegalArgumentException("Time period must be more than 0 milliseconds; Invalid Time period: " + timePeriod); } if (millis > Integer.MAX_VALUE) { throw new IllegalArgumentException("Timeout is too long; cannot be greater than " + Integer.MAX_VALUE + " milliseconds"); } this.communicationsTimeout = timePeriod; } catch (final Exception e) { - throw new IllegalArgumentException("Invalid Time Period: " + timePeriod); + throw new IllegalArgumentException("Invalid Time period: " + timePeriod); } } @@ -1233,7 +1233,7 @@ public void run() { public void setYieldDuration(final String yieldDuration) { // verify the syntax if (!FormatUtils.TIME_DURATION_PATTERN.matcher(yieldDuration).matches()) { - throw new IllegalArgumentException("Improperly formatted Time Period; should be of syntax where " + throw new IllegalArgumentException("Improperly formatted Time period; should be of syntax where " + " is a positive integer and unit is one of the valid Time Units, such as nanos, millis, sec, min, hour, day"); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_ScheduledProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledProcessor.java similarity index 76% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_ScheduledProcessor.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledProcessor.java index a9cc4abbdad4..c4e0ad582974 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_ScheduledProcessor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledProcessor.java @@ -11,8 +11,8 @@ /** * Dummy processor to test @DefaultSchedule annotation */ -@DefaultSchedule(ConcurrentTasks = 5, Strategy = SchedulingStrategy.CRON_DRIVEN,Period = "0 0 0 1/1 * ?") -public class Dummy_ScheduledProcessor extends AbstractProcessor { +@DefaultSchedule(concurrentTasks = 5, strategy = SchedulingStrategy.CRON_DRIVEN, period = "0 0 0 1/1 * ?") +public class DummyScheduledProcessor extends AbstractProcessor { @Override public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_SettingsProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java similarity index 77% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_SettingsProcessor.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java index 00b980e9d1be..a36e40c8f844 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/Dummy_SettingsProcessor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java @@ -10,8 +10,8 @@ /** * Dummy Processor to test @DefaultSettings annotation */ -@DefaultSettings(YieldDuration = "5 sec", PenaltyDuration = "1 min", LogLevel = LogLevel.DEBUG) -public class Dummy_SettingsProcessor extends AbstractProcessor { +@DefaultSettings(yieldDuration = "5 sec", penaltyDuration = "1 min", logLevel = LogLevel.DEBUG) +public class DummySettingsProcessor extends AbstractProcessor { @Override public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java index b25e435fca2a..32f813584619 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java @@ -47,7 +47,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.logging.LogLevel; -import org.apache.nifi.processor.*; +import org.apache.nifi.processor.Relationship; import org.apache.nifi.provenance.MockProvenanceRepository; import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.reporting.BulletinRepository; @@ -74,7 +74,7 @@ public class TestFlowController { @Before public void setup() { - System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, FlowController.class.getResource("/nifi.properties").getFile()); + System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFlowController.class.getResource("/nifi.properties").getFile()); flowFileEventRepo = Mockito.mock(FlowFileEventRepository.class); auditService = Mockito.mock(AuditService.class); @@ -332,7 +332,7 @@ public void testCreateMissingControllerService() throws ProcessorInstantiationEx @Test public void testProcessorDefaultScheduleAnnotation() throws ProcessorInstantiationException,ClassNotFoundException,InstantiationException,IllegalAccessException { - ProcessorNode p_scheduled = controller.createProcessor(Dummy_ScheduledProcessor.class.getName(),"1234-ScheduledProcessor"); + ProcessorNode p_scheduled = controller.createProcessor(DummyScheduledProcessor.class.getName(),"1234-ScheduledProcessor"); assertEquals(5,p_scheduled.getMaxConcurrentTasks()); assertEquals(SchedulingStrategy.CRON_DRIVEN,p_scheduled.getSchedulingStrategy()); assertEquals("0 0 0 1/1 * ?",p_scheduled.getSchedulingPeriod()); @@ -344,7 +344,7 @@ public void testProcessorDefaultScheduleAnnotation() throws ProcessorInstantiati @Test public void testProcessorDefaultSettingsAnnotation() throws ProcessorInstantiationException,ClassNotFoundException { - ProcessorNode p_settings = controller.createProcessor(Dummy_SettingsProcessor.class.getName(),"1234-SettingsProcessor"); + ProcessorNode p_settings = controller.createProcessor(DummySettingsProcessor.class.getName(),"1234-SettingsProcessor"); assertEquals("5 sec",p_settings.getYieldPeriod()); assertEquals("1 min",p_settings.getPenalizationPeriod()); assertEquals(LogLevel.DEBUG,p_settings.getBulletinLevel()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java index ee093deee220..7d7063bbae76 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java @@ -67,7 +67,7 @@ public class ProcessorAuditor extends NiFiAuditor { private static final String ANNOTATION_DATA = "Annotation Data"; private static final String AUTO_TERMINATED_RELATIONSHIPS = "Auto Terminated Relationships"; private static final String SCHEDULING_PERIOD = "Run Schedule"; - private static final String SCHEDULING_STRATEGY = "Scheduling Strategy"; + private static final String SCHEDULING_STRATEGY = "Scheduling strategy"; /** * Audits the creation of processors via createProcessor(). diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java index 6a6f175dd8be..73b5a43a4b4d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java @@ -264,9 +264,9 @@ private List validateProposedConfiguration(final ProcessorNode processor try { new CronExpression(config.getSchedulingPeriod()); } catch (final ParseException pe) { - throw new IllegalArgumentException(String.format("Scheduling Period '%s' is not a valid cron expression: %s", config.getSchedulingPeriod(), pe.getMessage())); + throw new IllegalArgumentException(String.format("Scheduling period '%s' is not a valid cron expression: %s", config.getSchedulingPeriod(), pe.getMessage())); } catch (final Exception e) { - throw new IllegalArgumentException("Scheduling Period is not a valid cron expression: " + config.getSchedulingPeriod()); + throw new IllegalArgumentException("Scheduling period is not a valid cron expression: " + config.getSchedulingPeriod()); } break; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java index ac3d9d556de8..a9d9f63cf04f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java @@ -179,9 +179,9 @@ private List validateProposedConfiguration(final ReportingTaskNode repor try { new CronExpression(reportingTaskDTO.getSchedulingPeriod()); } catch (final ParseException pe) { - throw new IllegalArgumentException(String.format("Scheduling Period '%s' is not a valid cron expression: %s", reportingTaskDTO.getSchedulingPeriod(), pe.getMessage())); + throw new IllegalArgumentException(String.format("Scheduling period '%s' is not a valid cron expression: %s", reportingTaskDTO.getSchedulingPeriod(), pe.getMessage())); } catch (final Exception e) { - throw new IllegalArgumentException("Scheduling Period is not a valid cron expression: " + reportingTaskDTO.getSchedulingPeriod()); + throw new IllegalArgumentException("Scheduling period is not a valid cron expression: " + reportingTaskDTO.getSchedulingPeriod()); } break; } diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java index ecb24aaeeecb..45f1c46a4f5a 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java @@ -114,8 +114,8 @@ public String toString() { .build(); public static final PropertyDescriptor KERBEROS_RELOGIN_PERIOD = new PropertyDescriptor.Builder() - .name("Kerberos Relogin Period").required(false) - .description("Period of time which should pass before attempting a kerberos relogin") + .name("Kerberos Relogin period").required(false) + .description("period of time which should pass before attempting a kerberos relogin") .defaultValue("4 hours") .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java index 3a0cb48a6a34..687bc27b2766 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java @@ -98,7 +98,7 @@ public class PutHDFS extends AbstractHadoopProcessor { // properties public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder() - .name("Conflict Resolution Strategy") + .name("Conflict Resolution strategy") .description("Indicates what should happen when a file with the same name already exists in the output directory") .required(true) .defaultValue(FAIL_RESOLUTION) diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java index f5d11f1e4f42..6bef4d490a14 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java @@ -74,7 +74,7 @@ public abstract class AbstractPutHBase extends AbstractProcessor { "Stores the value of the rows id as a binary byte array. It expects that the row id is a binary formated string."); static final PropertyDescriptor ROW_ID_ENCODING_STRATEGY = new PropertyDescriptor.Builder() - .name("Row Identifier Encoding Strategy") + .name("Row Identifier Encoding strategy") .description("Specifies the data type of Row ID used when inserting data into HBase. The default behaviror is" + " to convert the row id to a UTF-8 byte array. Choosing Binary will convert a binary formatted string" + " to the correct byte[] representation. The Binary option should be used if you are using Binary row" + diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java index 1294d9b164b7..057f9ad42c5d 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java @@ -58,7 +58,7 @@ @CapabilityDescription("Adds rows to HBase based on the contents of incoming JSON documents. Each FlowFile must contain a single " + "UTF-8 encoded JSON document, and any FlowFiles where the root element is not a single document will be routed to failure. " + "Each JSON field name and value will become a column qualifier and value of the HBase row. Any fields with a null value " + - "will be skipped, and fields with a complex value will be handled according to the Complex Field Strategy. " + + "will be skipped, and fields with a complex value will be handled according to the Complex Field strategy. " + "The row id can be specified either directly on the processor through the Row Identifier property, or can be extracted from the JSON " + "document by specifying the Row Identifier Field Name property. This processor will hold the contents of all FlowFiles for the given batch " + "in memory at one time.") @@ -82,7 +82,7 @@ public class PutHBaseJSON extends AbstractPutHBase { protected static final AllowableValue COMPLEX_FIELD_TEXT = new AllowableValue(TEXT_VALUE, TEXT_VALUE, "Use the string representation of the complex field as the value of the given column."); protected static final PropertyDescriptor COMPLEX_FIELD_STRATEGY = new PropertyDescriptor.Builder() - .name("Complex Field Strategy") + .name("Complex Field strategy") .description("Indicates how to handle complex fields, i.e. fields that do not have a single text value.") .expressionLanguageSupported(false) .required(true) @@ -98,7 +98,7 @@ public class PutHBaseJSON extends AbstractPutHBase { "Stores the value of each field as the byte representation of the type derived from the JSON."); protected static final PropertyDescriptor FIELD_ENCODING_STRATEGY = new PropertyDescriptor.Builder() - .name("Field Encoding Strategy") + .name("Field Encoding strategy") .description(("Indicates how to store the value of each field in HBase. The default behavior is to convert each value from the " + "JSON to a String, and store the UTF-8 bytes. Choosing Bytes will interpret the type of each field from " + "the JSON, and convert the value to the byte representation of that type, meaning an integer will be stored as the " + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index 4dc8d189a5cc..f59dc75ddbaa 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -121,7 +121,7 @@ public class PutKafka extends AbstractKafkaProcessor { .expressionLanguageSupported(true) .build(); static final PropertyDescriptor PARTITION_STRATEGY = new PropertyDescriptor.Builder() - .name("Partition Strategy") + .name("Partition strategy") .description("Specifies how messages should be partitioned when sent to Kafka") .allowableValues(ROUND_ROBIN_PARTITIONING, RANDOM_PARTITIONING, USER_DEFINED_PARTITIONING) .defaultValue(ROUND_ROBIN_PARTITIONING.getValue()) @@ -413,7 +413,7 @@ protected Collection customValidate(final ValidationContext va if (partitionStrategy.equalsIgnoreCase(USER_DEFINED_PARTITIONING.getValue()) && !validationContext.getProperty(PARTITION).isSet()) { results.add(new ValidationResult.Builder().subject("Partition").valid(false) - .explanation("The property must be set when configured to use the User-Defined Partitioning Strategy") + .explanation("The property must be set when configured to use the User-Defined Partitioning strategy") .build()); } return results; diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java index fdef62021df1..deb6c47d7a24 100644 --- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java +++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java @@ -101,7 +101,7 @@ public final void onConfigured(final LoginIdentityProviderConfigurationContext c } // authentication strategy - final String rawAuthenticationStrategy = configurationContext.getProperty("Authentication Strategy"); + final String rawAuthenticationStrategy = configurationContext.getProperty("Authentication strategy"); final LdapAuthenticationStrategy authenticationStrategy; try { authenticationStrategy = LdapAuthenticationStrategy.valueOf(rawAuthenticationStrategy); @@ -185,7 +185,7 @@ public final void onConfigured(final LoginIdentityProviderConfigurationContext c } // referrals - final String rawReferralStrategy = configurationContext.getProperty("Referral Strategy"); + final String rawReferralStrategy = configurationContext.getProperty("Referral strategy"); final ReferralStrategy referralStrategy; try { diff --git a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java index 4919e61a2c64..9ddd00b1bda8 100644 --- a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java +++ b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java @@ -121,7 +121,7 @@ public class GetSplunk extends AbstractProcessor { "Search based on the time the event was indexed in Splunk."); public static final PropertyDescriptor TIME_FIELD_STRATEGY = new PropertyDescriptor.Builder() - .name("Time Field Strategy") + .name("Time Field strategy") .description("Indicates whether to search by the time attached to the event, or by the time the event was indexed in Splunk.") .allowableValues(EVENT_TIME_VALUE, INDEX_TIME_VALUE) .defaultValue(EVENT_TIME_VALUE.getValue()) @@ -136,7 +136,7 @@ public class GetSplunk extends AbstractProcessor { "The the time range provided through the Earliest Time and Latest Time properties will be used."); public static final PropertyDescriptor TIME_RANGE_STRATEGY = new PropertyDescriptor.Builder() - .name("Time Range Strategy") + .name("Time Range strategy") .description("Indicates how to apply time ranges to each execution of the query. Selecting a managed option " + "allows the processor to apply a time range from the last execution time to the current execution time. " + "When using , an earliest time will not be applied on the first execution, and thus all " + @@ -150,14 +150,14 @@ public class GetSplunk extends AbstractProcessor { public static final PropertyDescriptor EARLIEST_TIME = new PropertyDescriptor.Builder() .name("Earliest Time") - .description("The value to use for the earliest time when querying. Only used with a Time Range Strategy of Provided. " + + .description("The value to use for the earliest time when querying. Only used with a Time Range strategy of Provided. " + "See Splunk's documentation on Search Time Modifiers for guidance in populating this field.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) .build(); public static final PropertyDescriptor LATEST_TIME = new PropertyDescriptor.Builder() .name("Latest Time") - .description("The value to use for the latest time when querying. Only used with a Time Range Strategy of Provided. " + + .description("The value to use for the latest time when querying. Only used with a Time Range strategy of Provided. " + "See Splunk's documentation on Search Time Modifiers for guidance in populating this field.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java index 8c1f9bdf136c..8a9bb4feff39 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java @@ -62,7 +62,7 @@ @InputRequirement(Requirement.INPUT_REQUIRED) @TriggerWhenAnyDestinationAvailable @Tags({"distribute", "load balance", "route", "round robin", "weighted"}) -@CapabilityDescription("Distributes FlowFiles to downstream processors based on a Distribution Strategy. If using the Round Robin " +@CapabilityDescription("Distributes FlowFiles to downstream processors based on a Distribution strategy. If using the Round Robin " + "strategy, the default is to assign each destination a weighting of 1 (evenly distributed). However, optional properties" + "can be added to the change this; adding a property with the name '5' and value '10' means that the relationship with name " + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") @@ -70,7 +70,7 @@ + "property with the name '5' and value '10' means that the relationship with name " + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") @DynamicRelationship(name = "A number 1..", description = "FlowFiles are sent to this relationship per the " - + "") + + "") public class DistributeLoad extends AbstractProcessor { public static final String STRATEGY_ROUND_ROBIN = "round robin"; @@ -85,7 +85,7 @@ public class DistributeLoad extends AbstractProcessor { .defaultValue("1") .build(); public static final PropertyDescriptor DISTRIBUTION_STRATEGY = new PropertyDescriptor.Builder() - .name("Distribution Strategy") + .name("Distribution strategy") .description("Determines how the load will be distributed. If using Round Robin, will not distribute any FlowFiles unless all " + "destinations can accept FlowFiles; when using Next Available, will distribute FlowFiles as long as at least 1 " + "destination can accept FlowFiles.") @@ -226,13 +226,13 @@ protected Collection customValidate(ValidationContext validati PropertyValue propDesc = validationContext.getProperty(HOSTNAMES); if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) { results.add(new ValidationResult.Builder().subject(HOSTNAMES.getName()) - .explanation("Must specify Hostnames when using 'Load Distribution Strategy'").valid(false).build()); + .explanation("Must specify Hostnames when using 'Load Distribution strategy'").valid(false).build()); } propDesc = validationContext.getProperty(LOAD_DISTRIBUTION_SERVICE_TEMPLATE); if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) { results.add(new ValidationResult.Builder() .subject(LOAD_DISTRIBUTION_SERVICE_TEMPLATE.getName()) - .explanation("Must specify 'Load Distribution Service ID' when using 'Load Distribution Strategy'") + .explanation("Must specify 'Load Distribution Service ID' when using 'Load Distribution strategy'") .valid(false).build()); } if (results.isEmpty()) { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java index 04675c9e0e61..2738ded47597 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java @@ -78,7 +78,7 @@ public class FetchFile extends AbstractProcessor { .required(true) .build(); static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder() - .name("Completion Strategy") + .name("Completion strategy") .description("Specifies what to do with the original file on the file system once it has been pulled into NiFi") .expressionLanguageSupported(false) .allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE) @@ -87,15 +87,15 @@ public class FetchFile extends AbstractProcessor { .build(); static final PropertyDescriptor MOVE_DESTINATION_DIR = new PropertyDescriptor.Builder() .name("Move Destination Directory") - .description("The directory to the move the original file to once it has been fetched from the file system. This property is ignored unless the Completion Strategy is set to \"Move File\". " + .description("The directory to the move the original file to once it has been fetched from the file system. This property is ignored unless the Completion strategy is set to \"Move File\". " + "If the directory does not exist, it will be created.") .expressionLanguageSupported(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) .build(); static final PropertyDescriptor CONFLICT_STRATEGY = new PropertyDescriptor.Builder() - .name("Move Conflict Strategy") - .description("If Completion Strategy is set to Move File and a file already exists in the destination directory with the same name, this property specifies " + .name("Move Conflict strategy") + .description("If Completion strategy is set to Move File and a file already exists in the destination directory with the same name, this property specifies " + "how that naming conflict should be resolved") .allowableValues(CONFLICT_RENAME, CONFLICT_REPLACE, CONFLICT_KEEP_INTACT, CONFLICT_FAIL) .defaultValue(CONFLICT_RENAME.getValue()) @@ -209,7 +209,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final File targetDir = new File(targetDirectoryName); if (COMPLETION_MOVE.getValue().equalsIgnoreCase(completionStrategy)) { if (targetDir.exists() && (!isWritable(targetDir) || !isDirectory(targetDir))) { - getLogger().error("Could not fetch file {} from file system for {} because Completion Strategy is configured to move the original file to {}, " + getLogger().error("Could not fetch file {} from file system for {} because Completion strategy is configured to move the original file to {}, " + "but that is not a directory or user {} does not have permissions to write to that directory", new Object[] {file, flowFile, targetDir, user}); session.transfer(flowFile, REL_FAILURE); @@ -221,8 +221,8 @@ public void onTrigger(final ProcessContext context, final ProcessSession session if (CONFLICT_FAIL.getValue().equalsIgnoreCase(conflictStrategy)) { final File targetFile = new File(targetDir, file.getName()); if (targetFile.exists()) { - getLogger().error("Could not fetch file {} from file system for {} because Completion Strategy is configured to move the original file to {}, " - + "but a file with name {} already exists in that directory and the Move Conflict Strategy is configured for failure", + getLogger().error("Could not fetch file {} from file system for {} because Completion strategy is configured to move the original file to {}, " + + "but a file with name {} already exists in that directory and the Move Conflict strategy is configured for failure", new Object[] {file, flowFile, targetDir, file.getName()}); session.transfer(flowFile, REL_FAILURE); return; @@ -243,12 +243,12 @@ public void onTrigger(final ProcessContext context, final ProcessSession session session.getProvenanceReporter().modifyContent(flowFile, "Replaced content of FlowFile with contents of " + file.toURI(), stopWatch.getElapsed(TimeUnit.MILLISECONDS)); session.transfer(flowFile, REL_SUCCESS); - // It is critical that we commit the session before we perform the Completion Strategy. Otherwise, we could have a case where we + // It is critical that we commit the session before we perform the Completion strategy. Otherwise, we could have a case where we // ingest the file, delete/move the file, and then NiFi is restarted before the session is committed. That would result in data loss. - // As long as we commit the session right here, before we perform the Completion Strategy, we are safe. + // As long as we commit the session right here, before we perform the Completion strategy, we are safe. session.commit(); - // Attempt to perform the Completion Strategy action + // Attempt to perform the Completion strategy action Exception completionFailureException = null; if (COMPLETION_DELETE.getValue().equalsIgnoreCase(completionStrategy)) { // convert to path and use Files.delete instead of file.delete so that if we fail, we know why diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java index a7ae5ef05943..9db75b24b747 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java @@ -92,8 +92,8 @@ public abstract class FetchFileTransfer extends AbstractProcessor { .expressionLanguageSupported(true) .build(); static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder() - .name("Completion Strategy") - .description("Specifies what to do with the original file on the server once it has been pulled into NiFi. If the Completion Strategy fails, a warning will be " + .name("Completion strategy") + .description("Specifies what to do with the original file on the server once it has been pulled into NiFi. If the Completion strategy fails, a warning will be " + "logged but the data will still be transferred.") .expressionLanguageSupported(false) .allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE) @@ -103,7 +103,7 @@ public abstract class FetchFileTransfer extends AbstractProcessor { static final PropertyDescriptor MOVE_DESTINATION_DIR = new PropertyDescriptor.Builder() .name("Move Destination Directory") .description("The directory on the remote server to the move the original file to once it has been ingested into NiFi. " - + "This property is ignored unless the Completion Strategy is set to \"Move File\". The specified directory must already exist on" + + "This property is ignored unless the Completion strategy is set to \"Move File\". The specified directory must already exist on" + "the remote system, or the rename will fail.") .expressionLanguageSupported(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index 545559ea8549..2cf27e383421 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -95,19 +95,19 @@ + "It is recommended that the Processor be configured with only a single incoming connection, as Group of FlowFiles will not be " + "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 property is set to Defragment. " + @ReadsAttribute(attribute = "fragment.identifier", description = "Applicable only if the 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 property is set to Defragment. " + @ReadsAttribute(attribute = "fragment.index", description = "Applicable only if the 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 " + + "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 property is set to Defragment. This " + @ReadsAttribute(attribute = "fragment.count", description = "Applicable only if the 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."), - @ReadsAttribute(attribute = "segment.original.filename", description = "Applicable only if the property is set to Defragment. " + @ReadsAttribute(attribute = "segment.original.filename", description = "Applicable only if the 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."), @@ -207,7 +207,7 @@ public class MergeContent extends BinFiles { public static final String MERGE_BIN_AGE_ATTRIBUTE = "merge.bin.age"; public static final PropertyDescriptor MERGE_STRATEGY = new PropertyDescriptor.Builder() - .name("Merge Strategy") + .name("Merge strategy") .description("Specifies the algorithm used to merge content. The 'Defragment' algorithm combines fragments that are associated by " + "attributes back into a single cohesive FlowFile. The 'Bin-Packing Algorithm' generates a FlowFile populated by arbitrarily " + "chosen FlowFiles") @@ -224,7 +224,7 @@ public class MergeContent extends BinFiles { .build(); public static final PropertyDescriptor ATTRIBUTE_STRATEGY = new PropertyDescriptor.Builder() .required(true) - .name("Attribute Strategy") + .name("Attribute strategy") .description("Determines which FlowFile attributes should be added to the bundle. If 'Keep All Unique Attributes' is selected, any " + "attribute on any FlowFile that gets bundled will be kept unless its value conflicts with the value from another FlowFile. " + "If 'Keep Only Common Attributes' is selected, only the attributes that exist on all FlowFiles in the bundle, with the same " @@ -245,7 +245,7 @@ public class MergeContent extends BinFiles { public static final PropertyDescriptor DELIMITER_STRATEGY = new PropertyDescriptor.Builder() .required(true) - .name("Delimiter Strategy") + .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) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java index 8ba328817dd5..a625c9a1adb7 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java @@ -83,7 +83,7 @@ public class PutFile extends AbstractProcessor { .addValidator(StandardValidators.INTEGER_VALIDATOR) .build(); public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder() - .name("Conflict Resolution Strategy") + .name("Conflict Resolution strategy") .description("Indicates what should happen when a file with the same name already exists in the output directory") .required(true) .defaultValue(FAIL_RESOLUTION) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java index a542048df7a3..8db641c58b3c 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java @@ -116,7 +116,7 @@ public class ReplaceText extends AbstractProcessor { .build(); public static final PropertyDescriptor REPLACEMENT_VALUE = new PropertyDescriptor.Builder() .name("Replacement Value") - .description("The value to insert using the 'Replacement Strategy'. Using \"Regex Replace\" back-references to Regular Expression capturing groups " + .description("The value to insert using the 'Replacement strategy'. Using \"Regex Replace\" back-references to Regular Expression capturing groups " + "are supported, but back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value. " + "Back References may also be referenced using the Expression Language, as '$1', '$2', etc. The single-tick marks MUST be included, as these variables are " + "not \"Standard\" attribute names (attribute names must be quoted unless they contain only numbers, letters, and _).") @@ -139,13 +139,13 @@ public class ReplaceText extends AbstractProcessor { + "the FlowFile will be routed to 'failure'. " + "In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value " + "of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. " - + "This value is ignored if the property is set to one of: Append, Prepend, Always Replace") + + "This value is ignored if the property is set to one of: Append, Prepend, Always Replace") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .defaultValue("1 MB") .build(); public static final PropertyDescriptor REPLACEMENT_STRATEGY = new PropertyDescriptor.Builder() - .name("Replacement Strategy") + .name("Replacement strategy") .description("The strategy for how and what to replace within the FlowFile's text content.") .allowableValues(PREPEND, APPEND, REGEX_REPLACE, LITERAL_REPLACE, ALWAYS_REPLACE) .defaultValue(REGEX_REPLACE.getValue()) @@ -153,7 +153,7 @@ public class ReplaceText extends AbstractProcessor { .build(); public static final PropertyDescriptor EVALUATION_MODE = new PropertyDescriptor.Builder() .name("Evaluation Mode") - .description("Run the 'Replacement Strategy' against each line separately (Line-by-Line) or buffer the entire file into memory (Entire Text) " + .description("Run the 'Replacement strategy' against each line separately (Line-by-Line) or buffer the entire file into memory (Entire Text) " + "and run against that.") .allowableValues(LINE_BY_LINE, ENTIRE_TEXT) .defaultValue(ENTIRE_TEXT) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java index 3a1ed9fe6cee..3dedd9a800a6 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java @@ -86,7 +86,7 @@ public class RouteOnAttribute extends AbstractProcessor { "Requires that at least one user-defined expression evaluate to 'true' for the FlowFile to be considered a match"); public static final PropertyDescriptor ROUTE_STRATEGY = new PropertyDescriptor.Builder() - .name("Routing Strategy") + .name("Routing strategy") .description("Specifies how to determine which relationship to use when evaluating the Expression Language") .required(true) .allowableValues(ROUTE_PROPERTY_NAME, ROUTE_ALL_MATCH, ROUTE_ANY_MATCHES) @@ -99,7 +99,7 @@ public class RouteOnAttribute extends AbstractProcessor { .build(); public static final Relationship REL_MATCH = new Relationship.Builder() .name("matched") - .description("FlowFiles will be routed to 'match' if one or all Expressions match, depending on the configuration of the Routing Strategy property") + .description("FlowFiles will be routed to 'match' if one or all Expressions match, depending on the configuration of the Routing strategy property") .build(); private AtomicReference> relationships = new AtomicReference<>(); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java index 01148f26e9ab..8ebebfa40a38 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java @@ -72,7 +72,7 @@ @InputRequirement(Requirement.INPUT_REQUIRED) @Tags({"attributes", "routing", "text", "regexp", "regex", "Regular Expression", "Expression Language", "csv", "filter", "logs", "delimited"}) @CapabilityDescription("Routes textual data based on a set of user-defined rules. Each line in an incoming FlowFile is compared against the values specified by user-defined Properties. " - + "The mechanism by which the text is compared to these user-defined properties is defined by the 'Matching Strategy'. The data is then routed according to these rules, routing " + + "The mechanism by which the text is compared to these user-defined properties is defined by the 'Matching strategy'. The data is then routed according to these rules, routing " + "each line of the text individually.") @DynamicProperty(name = "Relationship Name", value = "value to match against", description = "Routes data that matches the value specified in the Dynamic Property Value to the " + "Relationship specified in the Dynamic Property Key.") @@ -120,8 +120,8 @@ public class RouteText extends AbstractProcessor { + "'lineNo' (which is the line number being evaluated. This will be 1 for the first line, 2 for the second and so on)."); public static final PropertyDescriptor ROUTE_STRATEGY = new PropertyDescriptor.Builder() - .name("Routing Strategy") - .description("Specifies how to determine which Relationship(s) to use when evaluating the lines of incoming text against the 'Matching Strategy' and user-defined properties.") + .name("Routing strategy") + .description("Specifies how to determine which Relationship(s) to use when evaluating the lines of incoming text against the 'Matching strategy' and user-defined properties.") .required(true) .allowableValues(ROUTE_TO_MATCHING_PROPERTY_NAME, ROUTE_TO_MATCHED_WHEN_ALL_PROPERTIES_MATCH, ROUTE_TO_MATCHED_WHEN_ANY_PROPERTY_MATCHES) .defaultValue(ROUTE_TO_MATCHING_PROPERTY_NAME.getValue()) @@ -129,7 +129,7 @@ public class RouteText extends AbstractProcessor { .build(); public static final PropertyDescriptor MATCH_STRATEGY = new PropertyDescriptor.Builder() - .name("Matching Strategy") + .name("Matching strategy") .description("Specifies how to evaluate each line of incoming text against the user-defined properties.") .required(true) .allowableValues(SATISFIES_EXPRESSION, STARTS_WITH, ENDS_WITH, CONTAINS, EQUALS, MATCHES_REGULAR_EXPRESSION, CONTAINS_REGULAR_EXPRESSION) @@ -148,7 +148,7 @@ public class RouteText extends AbstractProcessor { static final PropertyDescriptor IGNORE_CASE = new PropertyDescriptor.Builder() .name("Ignore Case") .description("If true, capitalization will not be taken into account when comparing values. E.g., matching against 'HELLO' or 'hello' will have the same result. " - + "This property is ignored if the 'Matching Strategy' is set to 'Satisfies Expression'.") + + "This property is ignored if the 'Matching strategy' is set to 'Satisfies Expression'.") .expressionLanguageSupported(false) .allowableValues("true", "false") .defaultValue("false") diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java index ba2c9b86fa96..98535663bed3 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java @@ -196,7 +196,7 @@ public class TailFile extends AbstractProcessor { static final PropertyDescriptor ROLLING_STRATEGY = new PropertyDescriptor.Builder() .name("tailfile-rolling-strategy") - .displayName("Rolling Strategy") + .displayName("Rolling strategy") .description("Specifies if the files to tail have a fixed name or not.") .required(true) .allowableValues(FIXED_NAME, CHANGING_NAME) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java index 47880803160a..886dd132eaba 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java @@ -156,7 +156,7 @@ public class ValidateCsv extends AbstractProcessor { public static final PropertyDescriptor VALIDATION_STRATEGY = new PropertyDescriptor.Builder() .name("validate-csv-strategy") .displayName("Validation strategy") - .description("Strategy to apply when routing input files to output relationships.") + .description("strategy to apply when routing input files to output relationships.") .required(true) .defaultValue(VALIDATE_WHOLE_FLOWFILE.getValue()) .allowableValues(VALIDATE_LINES_INDIVIDUALLY, VALIDATE_WHOLE_FLOWFILE) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java index 7f6028dade1f..422b74f8d80d 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java @@ -78,7 +78,7 @@ * and a long value for the reportingInterval property will result in checking * the memory usage often so that notifications happen quickly but prevents * notifications from continually being generated. The format of this property - * is The Period format (e.g., 5 mins). + * is The period format (e.g., 5 mins). * * */ diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java b/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java index 44419b58f89f..b8b9419eee63 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java @@ -56,7 +56,7 @@ public abstract class DistributedCacheServer extends AbstractControllerService { .defaultValue("10000") .build(); public static final PropertyDescriptor EVICTION_POLICY = new PropertyDescriptor.Builder() - .name("Eviction Strategy") + .name("Eviction strategy") .description("Determines which strategy should be used to evict values from the cache to make room for new entries") .required(true) .allowableValues(EVICTION_STRATEGY_LFU, EVICTION_STRATEGY_LRU, EVICTION_STRATEGY_FIFO) From 4a8a763bfa3865c5d014f6d27f47cb45e293f385 Mon Sep 17 00:00:00 2001 From: Mathias Tiberghien Date: Sat, 8 Oct 2016 16:09:17 +0200 Subject: [PATCH 4/6] Removed '_' in Dummy classes lowercase methods in annotations Removed * in TestFlowController class import Replaced FlowController by TestFlowController in setup method of TestFlowController class --- .../annotation/behavior/TriggerWhenEmpty.java | 2 +- .../processor/util/StandardValidators.java | 4 ++-- .../nifi/processors/avro/SplitAvro.java | 8 ++++---- .../apache/nifi/controller/AbstractPort.java | 2 +- .../nifi/controller/StandardFunnel.java | 2 +- .../controller/StandardFlowFileQueue.java | 2 +- .../controller/StandardProcessorNode.java | 6 +++--- .../logging/ControllerServiceLogObserver.java | 4 ++-- .../nifi/logging/ProcessorLogObserver.java | 4 ++-- .../logging/ReportingTaskLogObserver.java | 4 ++-- .../remote/StandardRemoteProcessGroup.java | 6 +++--- .../apache/nifi/audit/ProcessorAuditor.java | 2 +- .../web/dao/impl/StandardProcessorDAO.java | 4 ++-- .../dao/impl/StandardReportingTaskDAO.java | 4 ++-- .../hadoop/AbstractHadoopProcessor.java | 4 ++-- .../nifi/processors/hadoop/PutHDFS.java | 2 +- .../apache/nifi/hbase/AbstractPutHBase.java | 2 +- .../org/apache/nifi/hbase/PutHBaseJSON.java | 6 +++--- .../nifi/processors/kafka/PutKafka.java | 4 ++-- .../org/apache/nifi/ldap/LdapProvider.java | 4 ++-- .../nifi/processors/splunk/GetSplunk.java | 8 ++++---- .../processors/standard/DistributeLoad.java | 10 +++++----- .../nifi/processors/standard/FetchFile.java | 20 +++++++++---------- .../standard/FetchFileTransfer.java | 6 +++--- .../processors/standard/MergeContent.java | 16 +++++++-------- .../nifi/processors/standard/PutFile.java | 2 +- .../nifi/processors/standard/ReplaceText.java | 8 ++++---- .../processors/standard/RouteOnAttribute.java | 4 ++-- .../nifi/processors/standard/RouteText.java | 10 +++++----- .../nifi/processors/standard/TailFile.java | 2 +- .../nifi/processors/standard/ValidateCsv.java | 2 +- .../apache/nifi/controller/MonitorMemory.java | 2 +- .../cache/server/DistributedCacheServer.java | 2 +- 33 files changed, 84 insertions(+), 84 deletions(-) diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java index bb23f22c9e0b..6fac03ba752a 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java @@ -31,7 +31,7 @@ *

* *

- * A Processor is scheduled to be triggered based on its configured Scheduling period + * A Processor is scheduled to be triggered based on its configured Scheduling Period * and Scheduling Strategy. However, when the scheduling period elapses, the Processor * will not be scheduled if it has no work to do. Normally, a Processor is said to have * work to do if one of the following circumstances is true: diff --git a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index 6eb3a8d7cb65..de1e57f59182 100644 --- a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -319,7 +319,7 @@ public ValidationResult validate(final String subject, final String input, final } if (input == null) { - return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time period cannot be null").build(); + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); } if (Pattern.compile(FormatUtils.TIME_DURATION_REGEX).matcher(input.toLowerCase()).matches()) { return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); @@ -652,7 +652,7 @@ public ValidationResult validate(final String subject, final String input, final } if (input == null) { - return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time period cannot be null").build(); + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); } final String lowerCase = input.toLowerCase(); final boolean validSyntax = pattern.matcher(lowerCase).matches(); diff --git a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java index b9de350536cf..e3eb6ec577f8 100644 --- a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java +++ b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java @@ -68,7 +68,7 @@ @SupportsBatching @Tags({ "avro", "split" }) @InputRequirement(Requirement.INPUT_REQUIRED) -@CapabilityDescription("Splits a binary encoded Avro datafile into smaller files based on the configured Output Size. The Output strategy determines if " + +@CapabilityDescription("Splits a binary encoded Avro datafile into smaller files based on the configured Output Size. The Output Strategy determines if " + "the smaller files will be Avro datafiles, or bare Avro records with metadata in the FlowFile attributes. The output will always be binary encoded.") @WritesAttributes({ @WritesAttribute(attribute = "fragment.identifier", @@ -85,7 +85,7 @@ public class SplitAvro extends AbstractProcessor { public static final AllowableValue RECORD_SPLIT = new AllowableValue(RECORD_SPLIT_VALUE, RECORD_SPLIT_VALUE, "Split at Record boundaries"); public static final PropertyDescriptor SPLIT_STRATEGY = new PropertyDescriptor.Builder() - .name("Split strategy") + .name("Split Strategy") .description("The strategy for splitting the incoming datafile. The Record strategy will read the incoming datafile by de-serializing each record.") .required(true) .allowableValues(RECORD_SPLIT) @@ -108,7 +108,7 @@ public class SplitAvro extends AbstractProcessor { public static final AllowableValue BARE_RECORD_OUTPUT = new AllowableValue(BARE_RECORD_OUTPUT_VALUE, BARE_RECORD_OUTPUT_VALUE, "Bare Avro records"); public static final PropertyDescriptor OUTPUT_STRATEGY = new PropertyDescriptor.Builder() - .name("Output strategy") + .name("Output Strategy") .description("Determines the format of the output. Either Avro Datafile, or bare record. Bare record output is only intended for use with systems " + "that already require it, and shouldn't be needed for normal use.") .required(true) @@ -118,7 +118,7 @@ public class SplitAvro extends AbstractProcessor { public static final PropertyDescriptor TRANSFER_METADATA = new PropertyDescriptor.Builder() .name("Transfer Metadata") - .description("Whether or not to transfer metadata from the parent datafile to the children. If the Output strategy is Bare Record, " + + .description("Whether or not to transfer metadata from the parent datafile to the children. If the Output Strategy is Bare Record, " + "then the metadata will be stored as FlowFile attributes, otherwise it will be in the Datafile header.") .required(true) .allowableValues("true", "false") diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java index eac5b914879f..581d209982b7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java @@ -465,7 +465,7 @@ public void setYieldPeriod(final String yieldPeriod) { public void setScheduldingPeriod(final String schedulingPeriod) { final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); if (schedulingNanos < 0) { - throw new IllegalArgumentException("Scheduling period must be positive"); + throw new IllegalArgumentException("Scheduling Period must be positive"); } this.schedulingPeriod.set(schedulingPeriod); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java index c56113ccfc75..c4cdf345f875 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java @@ -425,7 +425,7 @@ public void setYieldPeriod(final String yieldPeriod) { public void setScheduldingPeriod(final String schedulingPeriod) { final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); if (schedulingNanos < 0) { - throw new IllegalArgumentException("Scheduling period must be positive"); + throw new IllegalArgumentException("Scheduling Period must be positive"); } this.schedulingPeriod.set(schedulingPeriod); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java index b79dee1b8fdc..68af208983ec 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java @@ -802,7 +802,7 @@ public int getFlowFileExpiration(final TimeUnit timeUnit) { public void setFlowFileExpiration(final String flowExpirationPeriod) { final long millis = FormatUtils.getTimeDuration(flowExpirationPeriod, TimeUnit.MILLISECONDS); if (millis < 0) { - throw new IllegalArgumentException("FlowFile Expiration period must be positive"); + throw new IllegalArgumentException("FlowFile Expiration Period must be positive"); } expirationPeriod.set(new TimePeriod(flowExpirationPeriod, millis)); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index d75a22868011..8d891102633f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -430,7 +430,7 @@ public boolean isEventDrivenSupported() { } /** - * Updates the Scheduling strategy used for this Processor + * Updates the Scheduling Strategy used for this Processor * * @param schedulingStrategy * strategy @@ -479,7 +479,7 @@ public void setScheduldingPeriod(final String schedulingPeriod) { new CronExpression(schedulingPeriod); } catch (final Exception e) { throw new IllegalArgumentException( - "Scheduling period is not a valid cron expression: " + schedulingPeriod); + "Scheduling Period is not a valid cron expression: " + schedulingPeriod); } } break; @@ -488,7 +488,7 @@ public void setScheduldingPeriod(final String schedulingPeriod) { final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); if (schedulingNanos < 0) { - throw new IllegalArgumentException("Scheduling period must be positive"); + throw new IllegalArgumentException("Scheduling Period must be positive"); } this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos)); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java index de5b2aa55a4a..3ea432d40107 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java @@ -35,8 +35,8 @@ public ControllerServiceLogObserver(final BulletinRepository bulletinRepository, @Override public void onLogMessage(final LogMessage message) { - // Map logLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever - // the logLevel is (INFO and ERROR map directly and all others we will just accept as they are). + // Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever + // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are). final String bulletinLevel = message.getLevel() == LogLevel.WARN ? Severity.WARNING.name() : message.getLevel().toString(); final Bulletin bulletin = BulletinFactory.createBulletin(null, serviceNode.getIdentifier(), ComponentType.CONTROLLER_SERVICE, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java index a7eb03b7ff6d..62b90d63421c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java @@ -38,8 +38,8 @@ public ProcessorLogObserver(BulletinRepository bulletinRepository, ProcessorNode @Override public void onLogMessage(final LogMessage message) { - // Map logLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever - // the logLevel is (INFO and ERROR map directly and all others we will just accept as they are). + // Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever + // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are). final String bulletinLevel = (message.getLevel() == LogLevel.WARN) ? Severity.WARNING.name() : message.getLevel().toString(); bulletinRepository.addBulletin(BulletinFactory.createBulletin(processorNode, CATEGORY, bulletinLevel, message.getMessage())); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java index 235aeffb5c5b..f52bc1ccd61d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java @@ -34,8 +34,8 @@ public ReportingTaskLogObserver(final BulletinRepository bulletinRepository, fin @Override public void onLogMessage(final LogMessage message) { - // Map logLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever - // the logLevel is (INFO and ERROR map directly and all others we will just accept as they are). + // Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever + // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are). final String bulletinLevel = message.getLevel() == LogLevel.WARN ? Severity.WARNING.name() : message.getLevel().toString(); final Bulletin bulletin = BulletinFactory.createBulletin(null, taskNode.getIdentifier(), ComponentType.REPORTING_TASK, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java index c7dfe4756454..31bfd3dfba10 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java @@ -328,14 +328,14 @@ public void setCommunicationsTimeout(final String timePeriod) throws IllegalArgu try { final long millis = FormatUtils.getTimeDuration(timePeriod, TimeUnit.MILLISECONDS); if (millis <= 0) { - throw new IllegalArgumentException("Time period must be more than 0 milliseconds; Invalid Time period: " + timePeriod); + throw new IllegalArgumentException("Time Period must be more than 0 milliseconds; Invalid Time Period: " + timePeriod); } if (millis > Integer.MAX_VALUE) { throw new IllegalArgumentException("Timeout is too long; cannot be greater than " + Integer.MAX_VALUE + " milliseconds"); } this.communicationsTimeout = timePeriod; } catch (final Exception e) { - throw new IllegalArgumentException("Invalid Time period: " + timePeriod); + throw new IllegalArgumentException("Invalid Time Period: " + timePeriod); } } @@ -1233,7 +1233,7 @@ public void run() { public void setYieldDuration(final String yieldDuration) { // verify the syntax if (!FormatUtils.TIME_DURATION_PATTERN.matcher(yieldDuration).matches()) { - throw new IllegalArgumentException("Improperly formatted Time period; should be of syntax where " + throw new IllegalArgumentException("Improperly formatted Time Period; should be of syntax where " + " is a positive integer and unit is one of the valid Time Units, such as nanos, millis, sec, min, hour, day"); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java index 7d7063bbae76..ee093deee220 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java @@ -67,7 +67,7 @@ public class ProcessorAuditor extends NiFiAuditor { private static final String ANNOTATION_DATA = "Annotation Data"; private static final String AUTO_TERMINATED_RELATIONSHIPS = "Auto Terminated Relationships"; private static final String SCHEDULING_PERIOD = "Run Schedule"; - private static final String SCHEDULING_STRATEGY = "Scheduling strategy"; + private static final String SCHEDULING_STRATEGY = "Scheduling Strategy"; /** * Audits the creation of processors via createProcessor(). diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java index 73b5a43a4b4d..6a6f175dd8be 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java @@ -264,9 +264,9 @@ private List validateProposedConfiguration(final ProcessorNode processor try { new CronExpression(config.getSchedulingPeriod()); } catch (final ParseException pe) { - throw new IllegalArgumentException(String.format("Scheduling period '%s' is not a valid cron expression: %s", config.getSchedulingPeriod(), pe.getMessage())); + throw new IllegalArgumentException(String.format("Scheduling Period '%s' is not a valid cron expression: %s", config.getSchedulingPeriod(), pe.getMessage())); } catch (final Exception e) { - throw new IllegalArgumentException("Scheduling period is not a valid cron expression: " + config.getSchedulingPeriod()); + throw new IllegalArgumentException("Scheduling Period is not a valid cron expression: " + config.getSchedulingPeriod()); } break; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java index a9d9f63cf04f..ac3d9d556de8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java @@ -179,9 +179,9 @@ private List validateProposedConfiguration(final ReportingTaskNode repor try { new CronExpression(reportingTaskDTO.getSchedulingPeriod()); } catch (final ParseException pe) { - throw new IllegalArgumentException(String.format("Scheduling period '%s' is not a valid cron expression: %s", reportingTaskDTO.getSchedulingPeriod(), pe.getMessage())); + throw new IllegalArgumentException(String.format("Scheduling Period '%s' is not a valid cron expression: %s", reportingTaskDTO.getSchedulingPeriod(), pe.getMessage())); } catch (final Exception e) { - throw new IllegalArgumentException("Scheduling period is not a valid cron expression: " + reportingTaskDTO.getSchedulingPeriod()); + throw new IllegalArgumentException("Scheduling Period is not a valid cron expression: " + reportingTaskDTO.getSchedulingPeriod()); } break; } diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java index 45f1c46a4f5a..ecb24aaeeecb 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java @@ -114,8 +114,8 @@ public String toString() { .build(); public static final PropertyDescriptor KERBEROS_RELOGIN_PERIOD = new PropertyDescriptor.Builder() - .name("Kerberos Relogin period").required(false) - .description("period of time which should pass before attempting a kerberos relogin") + .name("Kerberos Relogin Period").required(false) + .description("Period of time which should pass before attempting a kerberos relogin") .defaultValue("4 hours") .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java index 687bc27b2766..3a0cb48a6a34 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java @@ -98,7 +98,7 @@ public class PutHDFS extends AbstractHadoopProcessor { // properties public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder() - .name("Conflict Resolution strategy") + .name("Conflict Resolution Strategy") .description("Indicates what should happen when a file with the same name already exists in the output directory") .required(true) .defaultValue(FAIL_RESOLUTION) diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java index 6bef4d490a14..f5d11f1e4f42 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/AbstractPutHBase.java @@ -74,7 +74,7 @@ public abstract class AbstractPutHBase extends AbstractProcessor { "Stores the value of the rows id as a binary byte array. It expects that the row id is a binary formated string."); static final PropertyDescriptor ROW_ID_ENCODING_STRATEGY = new PropertyDescriptor.Builder() - .name("Row Identifier Encoding strategy") + .name("Row Identifier Encoding Strategy") .description("Specifies the data type of Row ID used when inserting data into HBase. The default behaviror is" + " to convert the row id to a UTF-8 byte array. Choosing Binary will convert a binary formatted string" + " to the correct byte[] representation. The Binary option should be used if you are using Binary row" + diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java index 057f9ad42c5d..1294d9b164b7 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java @@ -58,7 +58,7 @@ @CapabilityDescription("Adds rows to HBase based on the contents of incoming JSON documents. Each FlowFile must contain a single " + "UTF-8 encoded JSON document, and any FlowFiles where the root element is not a single document will be routed to failure. " + "Each JSON field name and value will become a column qualifier and value of the HBase row. Any fields with a null value " + - "will be skipped, and fields with a complex value will be handled according to the Complex Field strategy. " + + "will be skipped, and fields with a complex value will be handled according to the Complex Field Strategy. " + "The row id can be specified either directly on the processor through the Row Identifier property, or can be extracted from the JSON " + "document by specifying the Row Identifier Field Name property. This processor will hold the contents of all FlowFiles for the given batch " + "in memory at one time.") @@ -82,7 +82,7 @@ public class PutHBaseJSON extends AbstractPutHBase { protected static final AllowableValue COMPLEX_FIELD_TEXT = new AllowableValue(TEXT_VALUE, TEXT_VALUE, "Use the string representation of the complex field as the value of the given column."); protected static final PropertyDescriptor COMPLEX_FIELD_STRATEGY = new PropertyDescriptor.Builder() - .name("Complex Field strategy") + .name("Complex Field Strategy") .description("Indicates how to handle complex fields, i.e. fields that do not have a single text value.") .expressionLanguageSupported(false) .required(true) @@ -98,7 +98,7 @@ public class PutHBaseJSON extends AbstractPutHBase { "Stores the value of each field as the byte representation of the type derived from the JSON."); protected static final PropertyDescriptor FIELD_ENCODING_STRATEGY = new PropertyDescriptor.Builder() - .name("Field Encoding strategy") + .name("Field Encoding Strategy") .description(("Indicates how to store the value of each field in HBase. The default behavior is to convert each value from the " + "JSON to a String, and store the UTF-8 bytes. Choosing Bytes will interpret the type of each field from " + "the JSON, and convert the value to the byte representation of that type, meaning an integer will be stored as the " + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index f59dc75ddbaa..4dc8d189a5cc 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -121,7 +121,7 @@ public class PutKafka extends AbstractKafkaProcessor { .expressionLanguageSupported(true) .build(); static final PropertyDescriptor PARTITION_STRATEGY = new PropertyDescriptor.Builder() - .name("Partition strategy") + .name("Partition Strategy") .description("Specifies how messages should be partitioned when sent to Kafka") .allowableValues(ROUND_ROBIN_PARTITIONING, RANDOM_PARTITIONING, USER_DEFINED_PARTITIONING) .defaultValue(ROUND_ROBIN_PARTITIONING.getValue()) @@ -413,7 +413,7 @@ protected Collection customValidate(final ValidationContext va if (partitionStrategy.equalsIgnoreCase(USER_DEFINED_PARTITIONING.getValue()) && !validationContext.getProperty(PARTITION).isSet()) { results.add(new ValidationResult.Builder().subject("Partition").valid(false) - .explanation("The property must be set when configured to use the User-Defined Partitioning strategy") + .explanation("The property must be set when configured to use the User-Defined Partitioning Strategy") .build()); } return results; diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java index deb6c47d7a24..fdef62021df1 100644 --- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java +++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java @@ -101,7 +101,7 @@ public final void onConfigured(final LoginIdentityProviderConfigurationContext c } // authentication strategy - final String rawAuthenticationStrategy = configurationContext.getProperty("Authentication strategy"); + final String rawAuthenticationStrategy = configurationContext.getProperty("Authentication Strategy"); final LdapAuthenticationStrategy authenticationStrategy; try { authenticationStrategy = LdapAuthenticationStrategy.valueOf(rawAuthenticationStrategy); @@ -185,7 +185,7 @@ public final void onConfigured(final LoginIdentityProviderConfigurationContext c } // referrals - final String rawReferralStrategy = configurationContext.getProperty("Referral strategy"); + final String rawReferralStrategy = configurationContext.getProperty("Referral Strategy"); final ReferralStrategy referralStrategy; try { diff --git a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java index 9ddd00b1bda8..4919e61a2c64 100644 --- a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java +++ b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java @@ -121,7 +121,7 @@ public class GetSplunk extends AbstractProcessor { "Search based on the time the event was indexed in Splunk."); public static final PropertyDescriptor TIME_FIELD_STRATEGY = new PropertyDescriptor.Builder() - .name("Time Field strategy") + .name("Time Field Strategy") .description("Indicates whether to search by the time attached to the event, or by the time the event was indexed in Splunk.") .allowableValues(EVENT_TIME_VALUE, INDEX_TIME_VALUE) .defaultValue(EVENT_TIME_VALUE.getValue()) @@ -136,7 +136,7 @@ public class GetSplunk extends AbstractProcessor { "The the time range provided through the Earliest Time and Latest Time properties will be used."); public static final PropertyDescriptor TIME_RANGE_STRATEGY = new PropertyDescriptor.Builder() - .name("Time Range strategy") + .name("Time Range Strategy") .description("Indicates how to apply time ranges to each execution of the query. Selecting a managed option " + "allows the processor to apply a time range from the last execution time to the current execution time. " + "When using , an earliest time will not be applied on the first execution, and thus all " + @@ -150,14 +150,14 @@ public class GetSplunk extends AbstractProcessor { public static final PropertyDescriptor EARLIEST_TIME = new PropertyDescriptor.Builder() .name("Earliest Time") - .description("The value to use for the earliest time when querying. Only used with a Time Range strategy of Provided. " + + .description("The value to use for the earliest time when querying. Only used with a Time Range Strategy of Provided. " + "See Splunk's documentation on Search Time Modifiers for guidance in populating this field.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) .build(); public static final PropertyDescriptor LATEST_TIME = new PropertyDescriptor.Builder() .name("Latest Time") - .description("The value to use for the latest time when querying. Only used with a Time Range strategy of Provided. " + + .description("The value to use for the latest time when querying. Only used with a Time Range Strategy of Provided. " + "See Splunk's documentation on Search Time Modifiers for guidance in populating this field.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java index 8a9bb4feff39..8c1f9bdf136c 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java @@ -62,7 +62,7 @@ @InputRequirement(Requirement.INPUT_REQUIRED) @TriggerWhenAnyDestinationAvailable @Tags({"distribute", "load balance", "route", "round robin", "weighted"}) -@CapabilityDescription("Distributes FlowFiles to downstream processors based on a Distribution strategy. If using the Round Robin " +@CapabilityDescription("Distributes FlowFiles to downstream processors based on a Distribution Strategy. If using the Round Robin " + "strategy, the default is to assign each destination a weighting of 1 (evenly distributed). However, optional properties" + "can be added to the change this; adding a property with the name '5' and value '10' means that the relationship with name " + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") @@ -70,7 +70,7 @@ + "property with the name '5' and value '10' means that the relationship with name " + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") @DynamicRelationship(name = "A number 1..", description = "FlowFiles are sent to this relationship per the " - + "") + + "") public class DistributeLoad extends AbstractProcessor { public static final String STRATEGY_ROUND_ROBIN = "round robin"; @@ -85,7 +85,7 @@ public class DistributeLoad extends AbstractProcessor { .defaultValue("1") .build(); public static final PropertyDescriptor DISTRIBUTION_STRATEGY = new PropertyDescriptor.Builder() - .name("Distribution strategy") + .name("Distribution Strategy") .description("Determines how the load will be distributed. If using Round Robin, will not distribute any FlowFiles unless all " + "destinations can accept FlowFiles; when using Next Available, will distribute FlowFiles as long as at least 1 " + "destination can accept FlowFiles.") @@ -226,13 +226,13 @@ protected Collection customValidate(ValidationContext validati PropertyValue propDesc = validationContext.getProperty(HOSTNAMES); if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) { results.add(new ValidationResult.Builder().subject(HOSTNAMES.getName()) - .explanation("Must specify Hostnames when using 'Load Distribution strategy'").valid(false).build()); + .explanation("Must specify Hostnames when using 'Load Distribution Strategy'").valid(false).build()); } propDesc = validationContext.getProperty(LOAD_DISTRIBUTION_SERVICE_TEMPLATE); if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) { results.add(new ValidationResult.Builder() .subject(LOAD_DISTRIBUTION_SERVICE_TEMPLATE.getName()) - .explanation("Must specify 'Load Distribution Service ID' when using 'Load Distribution strategy'") + .explanation("Must specify 'Load Distribution Service ID' when using 'Load Distribution Strategy'") .valid(false).build()); } if (results.isEmpty()) { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java index 2738ded47597..04675c9e0e61 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFile.java @@ -78,7 +78,7 @@ public class FetchFile extends AbstractProcessor { .required(true) .build(); static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder() - .name("Completion strategy") + .name("Completion Strategy") .description("Specifies what to do with the original file on the file system once it has been pulled into NiFi") .expressionLanguageSupported(false) .allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE) @@ -87,15 +87,15 @@ public class FetchFile extends AbstractProcessor { .build(); static final PropertyDescriptor MOVE_DESTINATION_DIR = new PropertyDescriptor.Builder() .name("Move Destination Directory") - .description("The directory to the move the original file to once it has been fetched from the file system. This property is ignored unless the Completion strategy is set to \"Move File\". " + .description("The directory to the move the original file to once it has been fetched from the file system. This property is ignored unless the Completion Strategy is set to \"Move File\". " + "If the directory does not exist, it will be created.") .expressionLanguageSupported(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) .build(); static final PropertyDescriptor CONFLICT_STRATEGY = new PropertyDescriptor.Builder() - .name("Move Conflict strategy") - .description("If Completion strategy is set to Move File and a file already exists in the destination directory with the same name, this property specifies " + .name("Move Conflict Strategy") + .description("If Completion Strategy is set to Move File and a file already exists in the destination directory with the same name, this property specifies " + "how that naming conflict should be resolved") .allowableValues(CONFLICT_RENAME, CONFLICT_REPLACE, CONFLICT_KEEP_INTACT, CONFLICT_FAIL) .defaultValue(CONFLICT_RENAME.getValue()) @@ -209,7 +209,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final File targetDir = new File(targetDirectoryName); if (COMPLETION_MOVE.getValue().equalsIgnoreCase(completionStrategy)) { if (targetDir.exists() && (!isWritable(targetDir) || !isDirectory(targetDir))) { - getLogger().error("Could not fetch file {} from file system for {} because Completion strategy is configured to move the original file to {}, " + getLogger().error("Could not fetch file {} from file system for {} because Completion Strategy is configured to move the original file to {}, " + "but that is not a directory or user {} does not have permissions to write to that directory", new Object[] {file, flowFile, targetDir, user}); session.transfer(flowFile, REL_FAILURE); @@ -221,8 +221,8 @@ public void onTrigger(final ProcessContext context, final ProcessSession session if (CONFLICT_FAIL.getValue().equalsIgnoreCase(conflictStrategy)) { final File targetFile = new File(targetDir, file.getName()); if (targetFile.exists()) { - getLogger().error("Could not fetch file {} from file system for {} because Completion strategy is configured to move the original file to {}, " - + "but a file with name {} already exists in that directory and the Move Conflict strategy is configured for failure", + getLogger().error("Could not fetch file {} from file system for {} because Completion Strategy is configured to move the original file to {}, " + + "but a file with name {} already exists in that directory and the Move Conflict Strategy is configured for failure", new Object[] {file, flowFile, targetDir, file.getName()}); session.transfer(flowFile, REL_FAILURE); return; @@ -243,12 +243,12 @@ public void onTrigger(final ProcessContext context, final ProcessSession session session.getProvenanceReporter().modifyContent(flowFile, "Replaced content of FlowFile with contents of " + file.toURI(), stopWatch.getElapsed(TimeUnit.MILLISECONDS)); session.transfer(flowFile, REL_SUCCESS); - // It is critical that we commit the session before we perform the Completion strategy. Otherwise, we could have a case where we + // It is critical that we commit the session before we perform the Completion Strategy. Otherwise, we could have a case where we // ingest the file, delete/move the file, and then NiFi is restarted before the session is committed. That would result in data loss. - // As long as we commit the session right here, before we perform the Completion strategy, we are safe. + // As long as we commit the session right here, before we perform the Completion Strategy, we are safe. session.commit(); - // Attempt to perform the Completion strategy action + // Attempt to perform the Completion Strategy action Exception completionFailureException = null; if (COMPLETION_DELETE.getValue().equalsIgnoreCase(completionStrategy)) { // convert to path and use Files.delete instead of file.delete so that if we fail, we know why diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java index 9db75b24b747..a7ae5ef05943 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java @@ -92,8 +92,8 @@ public abstract class FetchFileTransfer extends AbstractProcessor { .expressionLanguageSupported(true) .build(); static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder() - .name("Completion strategy") - .description("Specifies what to do with the original file on the server once it has been pulled into NiFi. If the Completion strategy fails, a warning will be " + .name("Completion Strategy") + .description("Specifies what to do with the original file on the server once it has been pulled into NiFi. If the Completion Strategy fails, a warning will be " + "logged but the data will still be transferred.") .expressionLanguageSupported(false) .allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE) @@ -103,7 +103,7 @@ public abstract class FetchFileTransfer extends AbstractProcessor { static final PropertyDescriptor MOVE_DESTINATION_DIR = new PropertyDescriptor.Builder() .name("Move Destination Directory") .description("The directory on the remote server to the move the original file to once it has been ingested into NiFi. " - + "This property is ignored unless the Completion strategy is set to \"Move File\". The specified directory must already exist on" + + "This property is ignored unless the Completion Strategy is set to \"Move File\". The specified directory must already exist on" + "the remote system, or the rename will fail.") .expressionLanguageSupported(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index 2cf27e383421..545559ea8549 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -95,19 +95,19 @@ + "It is recommended that the Processor be configured with only a single incoming connection, as Group of FlowFiles will not be " + "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 property is set to Defragment. " + @ReadsAttribute(attribute = "fragment.identifier", description = "Applicable only if the 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 property is set to Defragment. " + @ReadsAttribute(attribute = "fragment.index", description = "Applicable only if the 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 " + + "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 property is set to Defragment. This " + @ReadsAttribute(attribute = "fragment.count", description = "Applicable only if the 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."), - @ReadsAttribute(attribute = "segment.original.filename", description = "Applicable only if the property is set to Defragment. " + @ReadsAttribute(attribute = "segment.original.filename", description = "Applicable only if the 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."), @@ -207,7 +207,7 @@ public class MergeContent extends BinFiles { public static final String MERGE_BIN_AGE_ATTRIBUTE = "merge.bin.age"; public static final PropertyDescriptor MERGE_STRATEGY = new PropertyDescriptor.Builder() - .name("Merge strategy") + .name("Merge Strategy") .description("Specifies the algorithm used to merge content. The 'Defragment' algorithm combines fragments that are associated by " + "attributes back into a single cohesive FlowFile. The 'Bin-Packing Algorithm' generates a FlowFile populated by arbitrarily " + "chosen FlowFiles") @@ -224,7 +224,7 @@ public class MergeContent extends BinFiles { .build(); public static final PropertyDescriptor ATTRIBUTE_STRATEGY = new PropertyDescriptor.Builder() .required(true) - .name("Attribute strategy") + .name("Attribute Strategy") .description("Determines which FlowFile attributes should be added to the bundle. If 'Keep All Unique Attributes' is selected, any " + "attribute on any FlowFile that gets bundled will be kept unless its value conflicts with the value from another FlowFile. " + "If 'Keep Only Common Attributes' is selected, only the attributes that exist on all FlowFiles in the bundle, with the same " @@ -245,7 +245,7 @@ public class MergeContent extends BinFiles { public static final PropertyDescriptor DELIMITER_STRATEGY = new PropertyDescriptor.Builder() .required(true) - .name("Delimiter strategy") + .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) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java index a625c9a1adb7..8ba328817dd5 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java @@ -83,7 +83,7 @@ public class PutFile extends AbstractProcessor { .addValidator(StandardValidators.INTEGER_VALIDATOR) .build(); public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder() - .name("Conflict Resolution strategy") + .name("Conflict Resolution Strategy") .description("Indicates what should happen when a file with the same name already exists in the output directory") .required(true) .defaultValue(FAIL_RESOLUTION) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java index 8db641c58b3c..a542048df7a3 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java @@ -116,7 +116,7 @@ public class ReplaceText extends AbstractProcessor { .build(); public static final PropertyDescriptor REPLACEMENT_VALUE = new PropertyDescriptor.Builder() .name("Replacement Value") - .description("The value to insert using the 'Replacement strategy'. Using \"Regex Replace\" back-references to Regular Expression capturing groups " + .description("The value to insert using the 'Replacement Strategy'. Using \"Regex Replace\" back-references to Regular Expression capturing groups " + "are supported, but back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value. " + "Back References may also be referenced using the Expression Language, as '$1', '$2', etc. The single-tick marks MUST be included, as these variables are " + "not \"Standard\" attribute names (attribute names must be quoted unless they contain only numbers, letters, and _).") @@ -139,13 +139,13 @@ public class ReplaceText extends AbstractProcessor { + "the FlowFile will be routed to 'failure'. " + "In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value " + "of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. " - + "This value is ignored if the property is set to one of: Append, Prepend, Always Replace") + + "This value is ignored if the property is set to one of: Append, Prepend, Always Replace") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .defaultValue("1 MB") .build(); public static final PropertyDescriptor REPLACEMENT_STRATEGY = new PropertyDescriptor.Builder() - .name("Replacement strategy") + .name("Replacement Strategy") .description("The strategy for how and what to replace within the FlowFile's text content.") .allowableValues(PREPEND, APPEND, REGEX_REPLACE, LITERAL_REPLACE, ALWAYS_REPLACE) .defaultValue(REGEX_REPLACE.getValue()) @@ -153,7 +153,7 @@ public class ReplaceText extends AbstractProcessor { .build(); public static final PropertyDescriptor EVALUATION_MODE = new PropertyDescriptor.Builder() .name("Evaluation Mode") - .description("Run the 'Replacement strategy' against each line separately (Line-by-Line) or buffer the entire file into memory (Entire Text) " + .description("Run the 'Replacement Strategy' against each line separately (Line-by-Line) or buffer the entire file into memory (Entire Text) " + "and run against that.") .allowableValues(LINE_BY_LINE, ENTIRE_TEXT) .defaultValue(ENTIRE_TEXT) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java index 3dedd9a800a6..3a1ed9fe6cee 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java @@ -86,7 +86,7 @@ public class RouteOnAttribute extends AbstractProcessor { "Requires that at least one user-defined expression evaluate to 'true' for the FlowFile to be considered a match"); public static final PropertyDescriptor ROUTE_STRATEGY = new PropertyDescriptor.Builder() - .name("Routing strategy") + .name("Routing Strategy") .description("Specifies how to determine which relationship to use when evaluating the Expression Language") .required(true) .allowableValues(ROUTE_PROPERTY_NAME, ROUTE_ALL_MATCH, ROUTE_ANY_MATCHES) @@ -99,7 +99,7 @@ public class RouteOnAttribute extends AbstractProcessor { .build(); public static final Relationship REL_MATCH = new Relationship.Builder() .name("matched") - .description("FlowFiles will be routed to 'match' if one or all Expressions match, depending on the configuration of the Routing strategy property") + .description("FlowFiles will be routed to 'match' if one or all Expressions match, depending on the configuration of the Routing Strategy property") .build(); private AtomicReference> relationships = new AtomicReference<>(); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java index 8ebebfa40a38..01148f26e9ab 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java @@ -72,7 +72,7 @@ @InputRequirement(Requirement.INPUT_REQUIRED) @Tags({"attributes", "routing", "text", "regexp", "regex", "Regular Expression", "Expression Language", "csv", "filter", "logs", "delimited"}) @CapabilityDescription("Routes textual data based on a set of user-defined rules. Each line in an incoming FlowFile is compared against the values specified by user-defined Properties. " - + "The mechanism by which the text is compared to these user-defined properties is defined by the 'Matching strategy'. The data is then routed according to these rules, routing " + + "The mechanism by which the text is compared to these user-defined properties is defined by the 'Matching Strategy'. The data is then routed according to these rules, routing " + "each line of the text individually.") @DynamicProperty(name = "Relationship Name", value = "value to match against", description = "Routes data that matches the value specified in the Dynamic Property Value to the " + "Relationship specified in the Dynamic Property Key.") @@ -120,8 +120,8 @@ public class RouteText extends AbstractProcessor { + "'lineNo' (which is the line number being evaluated. This will be 1 for the first line, 2 for the second and so on)."); public static final PropertyDescriptor ROUTE_STRATEGY = new PropertyDescriptor.Builder() - .name("Routing strategy") - .description("Specifies how to determine which Relationship(s) to use when evaluating the lines of incoming text against the 'Matching strategy' and user-defined properties.") + .name("Routing Strategy") + .description("Specifies how to determine which Relationship(s) to use when evaluating the lines of incoming text against the 'Matching Strategy' and user-defined properties.") .required(true) .allowableValues(ROUTE_TO_MATCHING_PROPERTY_NAME, ROUTE_TO_MATCHED_WHEN_ALL_PROPERTIES_MATCH, ROUTE_TO_MATCHED_WHEN_ANY_PROPERTY_MATCHES) .defaultValue(ROUTE_TO_MATCHING_PROPERTY_NAME.getValue()) @@ -129,7 +129,7 @@ public class RouteText extends AbstractProcessor { .build(); public static final PropertyDescriptor MATCH_STRATEGY = new PropertyDescriptor.Builder() - .name("Matching strategy") + .name("Matching Strategy") .description("Specifies how to evaluate each line of incoming text against the user-defined properties.") .required(true) .allowableValues(SATISFIES_EXPRESSION, STARTS_WITH, ENDS_WITH, CONTAINS, EQUALS, MATCHES_REGULAR_EXPRESSION, CONTAINS_REGULAR_EXPRESSION) @@ -148,7 +148,7 @@ public class RouteText extends AbstractProcessor { static final PropertyDescriptor IGNORE_CASE = new PropertyDescriptor.Builder() .name("Ignore Case") .description("If true, capitalization will not be taken into account when comparing values. E.g., matching against 'HELLO' or 'hello' will have the same result. " - + "This property is ignored if the 'Matching strategy' is set to 'Satisfies Expression'.") + + "This property is ignored if the 'Matching Strategy' is set to 'Satisfies Expression'.") .expressionLanguageSupported(false) .allowableValues("true", "false") .defaultValue("false") diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java index 98535663bed3..ba2c9b86fa96 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java @@ -196,7 +196,7 @@ public class TailFile extends AbstractProcessor { static final PropertyDescriptor ROLLING_STRATEGY = new PropertyDescriptor.Builder() .name("tailfile-rolling-strategy") - .displayName("Rolling strategy") + .displayName("Rolling Strategy") .description("Specifies if the files to tail have a fixed name or not.") .required(true) .allowableValues(FIXED_NAME, CHANGING_NAME) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java index 886dd132eaba..47880803160a 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateCsv.java @@ -156,7 +156,7 @@ public class ValidateCsv extends AbstractProcessor { public static final PropertyDescriptor VALIDATION_STRATEGY = new PropertyDescriptor.Builder() .name("validate-csv-strategy") .displayName("Validation strategy") - .description("strategy to apply when routing input files to output relationships.") + .description("Strategy to apply when routing input files to output relationships.") .required(true) .defaultValue(VALIDATE_WHOLE_FLOWFILE.getValue()) .allowableValues(VALIDATE_LINES_INDIVIDUALLY, VALIDATE_WHOLE_FLOWFILE) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java index 422b74f8d80d..7f6028dade1f 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java @@ -78,7 +78,7 @@ * and a long value for the reportingInterval property will result in checking * the memory usage often so that notifications happen quickly but prevents * notifications from continually being generated. The format of this property - * is The period format (e.g., 5 mins). + * is The Period format (e.g., 5 mins). * * */ diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java b/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java index b8b9419eee63..44419b58f89f 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java @@ -56,7 +56,7 @@ public abstract class DistributedCacheServer extends AbstractControllerService { .defaultValue("10000") .build(); public static final PropertyDescriptor EVICTION_POLICY = new PropertyDescriptor.Builder() - .name("Eviction strategy") + .name("Eviction Strategy") .description("Determines which strategy should be used to evict values from the cache to make room for new entries") .required(true) .allowableValues(EVICTION_STRATEGY_LFU, EVICTION_STRATEGY_LRU, EVICTION_STRATEGY_FIFO) From d29b42e9ac69432e694f148b296907463d7237df Mon Sep 17 00:00:00 2001 From: Mathias Tiberghien Date: Mon, 10 Oct 2016 17:05:03 +0200 Subject: [PATCH 5/6] Added Licence headers to files: - DefaultSchedule.java - DefaultSettings.java - DummyScheduledProcessor.java - dummySettingsProcessor.java --- .../configuration/DefaultSchedule.java | 17 +++++++++++++++++ .../configuration/DefaultSettings.java | 17 +++++++++++++++++ .../controller/DummyScheduledProcessor.java | 17 +++++++++++++++++ .../nifi/controller/DummySettingsProcessor.java | 17 +++++++++++++++++ 4 files changed, 68 insertions(+) diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java index 10e6677c36b4..ff9125cd701a 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java @@ -1,3 +1,20 @@ +/* + * 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.annotation.configuration; import org.apache.nifi.scheduling.SchedulingStrategy; diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java index 5800b053ef61..62d9ae970f79 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java @@ -1,3 +1,20 @@ +/* + * 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.annotation.configuration; import java.lang.annotation.Documented; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledProcessor.java index c4e0ad582974..b8e469b2e207 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledProcessor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummyScheduledProcessor.java @@ -1,3 +1,20 @@ +/* + * 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.controller; import org.apache.nifi.annotation.configuration.DefaultSchedule; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java index a36e40c8f844..7338ffcc28bf 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java @@ -1,3 +1,20 @@ +/* + * 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.controller; import org.apache.nifi.annotation.configuration.DefaultSettings; From cda3e3704e0bc9c21e627591e9fb47b9f05a63ef Mon Sep 17 00:00:00 2001 From: Mathias Tiberghien Date: Thu, 13 Oct 2016 21:38:32 +0200 Subject: [PATCH 6/6] Change logLevel() method to bulletinLevel() in DefaultSettings Change applied to FlowController and DummySettingsProcessor --- .../apache/nifi/annotation/configuration/DefaultSettings.java | 4 ++-- .../main/java/org/apache/nifi/controller/FlowController.java | 2 +- .../org/apache/nifi/controller/DummySettingsProcessor.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java index 62d9ae970f79..d01972c8c724 100644 --- a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSettings.java @@ -28,7 +28,7 @@ /** *

* Marker interface that a Processor can use to configure the yield duration, the penalty duration and the bulletin log level. - * Note that the number of Concurrent tasks will be ignored if the annotion @TriggerSerialy is used + * Note that the number of Concurrent tasks will be ignored if the annotation @TriggerSerialy is used *

*/ @Documented @@ -38,5 +38,5 @@ public @interface DefaultSettings { String yieldDuration() default "1 sec"; String penaltyDuration() default "30 sec"; - LogLevel logLevel() default LogLevel.WARN; + LogLevel bulletinLevel() default LogLevel.WARN; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 41fa3d35d339..e9184824d173 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -1076,7 +1076,7 @@ public ProcessorNode createProcessor(final String type, String id, final boolean LOG.error(String.format("Error while setting penalty duration from DefaultSettings annotation:%s",ex.getMessage()),ex); } try { - procNode.setBulletinLevel(ds.logLevel()); + procNode.setBulletinLevel(ds.bulletinLevel()); } catch (Throwable ex) { LOG.error(String.format("Error while setting bulletin level from DefaultSettings annotation:%s",ex.getMessage()),ex); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java index 7338ffcc28bf..34c16afda346 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/DummySettingsProcessor.java @@ -27,7 +27,7 @@ /** * Dummy Processor to test @DefaultSettings annotation */ -@DefaultSettings(yieldDuration = "5 sec", penaltyDuration = "1 min", logLevel = LogLevel.DEBUG) +@DefaultSettings(yieldDuration = "5 sec", penaltyDuration = "1 min", bulletinLevel = LogLevel.DEBUG) public class DummySettingsProcessor extends AbstractProcessor { @Override public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {