From f0f175f9069de0bcff594e88b8043a7d5ebc5f40 Mon Sep 17 00:00:00 2001 From: Andrew Grande Date: Fri, 2 Jun 2017 15:28:30 -0400 Subject: [PATCH] Cleanup Azure Event Hub processors: - use name and displayName consistently - fixed user-facing typos - extracted a common superclass for DRY --- .../azure/eventhub/AbstractAzureEventHub.java | 35 +++++++++++++ .../azure/eventhub/GetAzureEventHub.java | 51 ++++++++----------- .../azure/eventhub/PutAzureEventHub.java | 31 +++-------- 3 files changed, 61 insertions(+), 56 deletions(-) create mode 100644 nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/AbstractAzureEventHub.java diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/AbstractAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/AbstractAzureEventHub.java new file mode 100644 index 000000000000..70f897618ee3 --- /dev/null +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/AbstractAzureEventHub.java @@ -0,0 +1,35 @@ +package org.apache.nifi.processors.azure.eventhub; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.util.StandardValidators; + +public abstract class AbstractAzureEventHub extends AbstractProcessor { + + static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder() + .name("event-hub-name") + .displayName("Event Hub Name") + .description("The name of the Azure Event Hub to pull messages from") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .required(true) + .build(); + + static final PropertyDescriptor NAMESPACE = new PropertyDescriptor.Builder() + .name("event-hub-namespace") + .displayName("Event Hub Namespace") + .description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to -ns") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(false) + .required(true) + .build(); + + static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder() + .name("shared-access-policy-primary-key") + .displayName("Shared Access Policy Primary Key") + .description("The primary key of the Event Hub Shared Access Policy") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(false) + .sensitive(true) + .required(true) + .build(); +} diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java index 69d558640bec..dcb3c9d8424f 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java @@ -31,7 +31,6 @@ import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; @@ -68,23 +67,11 @@ @WritesAttribute(attribute = "eventhub.name", description = "The name of the Event Hub from which the message was pulled"), @WritesAttribute(attribute = "eventhub.partition", description = "The name of the Azure Partition from which the message was pulled") }) -public class GetAzureEventHub extends AbstractProcessor { +public class GetAzureEventHub extends AbstractAzureEventHub { - static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder() - .name("Event Hub Name") - .description("The name of the Azure Event Hub to pull messages from") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .required(true) - .build(); - static final PropertyDescriptor NAMESPACE = new PropertyDescriptor.Builder() - .name("Event Hub Namespace") - .description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to -ns") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(false) - .required(true) - .build(); static final PropertyDescriptor SERVICE_BUS_ENDPOINT = new PropertyDescriptor.Builder() - .name("Service Bus Endpoint") + .name("service-bus-endpoint") + .displayName("Service Bus Endpoint") .description("To support Namespaces in non-standard Host URIs ( not .servicebus.windows.net, ie .servicebus.chinacloudapi.cn) select from the drop down acceptable options ") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(false) @@ -92,32 +79,29 @@ public class GetAzureEventHub extends AbstractProcessor { .defaultValue(".servicebus.windows.net") .required(true) .build(); + static final PropertyDescriptor ACCESS_POLICY = new PropertyDescriptor.Builder() - .name("Shared Access Policy Name") + .name("shared-access-listen-policy-name") + .displayName("Shared Access Listen Policy Name") .description("The name of the Event Hub Shared Access Policy. This Policy must have Listen permissions.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(false) .required(true) .build(); - static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder() - .name("Shared Access Policy Primary Key") - .description("The primary key of the Event Hub Shared Access Policy") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(false) - .sensitive(true) - .required(true) - .build(); static final PropertyDescriptor NUM_PARTITIONS = new PropertyDescriptor.Builder() - .name("Number of Event Hub Partitions") + .name("num-event-hub-partitions") + .displayName("Number of Event Hub Partitions") .description("The number of partitions that the Event Hub has. Only this number of partitions will be used, " + "so it is important to ensure that if the number of partitions changes that this value be updated. Otherwise, some messages may not be consumed.") .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .expressionLanguageSupported(false) .required(true) .build(); + static final PropertyDescriptor CONSUMER_GROUP = new PropertyDescriptor.Builder() - .name("Event Hub Consumer Group") + .name("event-hub-consumer-group") + .displayName("Event Hub Consumer Group") .description("The name of the Event Hub Consumer Group to use when pulling events") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(false) @@ -126,22 +110,27 @@ public class GetAzureEventHub extends AbstractProcessor { .build(); static final PropertyDescriptor ENQUEUE_TIME = new PropertyDescriptor.Builder() - .name("Event Hub Message Enqueue Time") + .name("event-hub-msg-enqueue-time") + .displayName("Event Hub Message Enqueue Time") .description("A timestamp (ISO-8061 Instant) formatted as YYYY-MM-DDThhmmss.sssZ (2016-01-01T01:01:01.000Z) from which messages " + "should have been enqueued in the EventHub to start reading from") .addValidator(StandardValidators.ISO8061_INSTANT_VALIDATOR) .expressionLanguageSupported(false) .required(false) .build(); + static final PropertyDescriptor RECEIVER_FETCH_SIZE = new PropertyDescriptor.Builder() - .name("Partition Recivier Fetch Size") + .name("partition-receiver-fetch-size") + .displayName("Partition Receiver Fetch Size") .description("The number of events that a receiver should fetch from an EventHubs partition before returning. Default(100)") .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .expressionLanguageSupported(false) .required(false) .build(); + static final PropertyDescriptor RECEIVER_FETCH_TIMEOUT = new PropertyDescriptor.Builder() - .name("Partiton Receiver Timeout (millseconds)") + .name("partition-receiver-timeout-ms") + .displayName("Partiton Receiver Timeout (millseconds)") .description("The amount of time a Partition Receiver should wait to receive the Fetch Size before returning. Default(60000)") .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) .expressionLanguageSupported(false) @@ -243,7 +232,7 @@ PartitionReceiver getReceiver(final ProcessContext context, final String partiti /** * This method is here to try and isolate the Azure related code as the PartitionReceiver cannot be mocked * with PowerMock due to it being final. Unfortunately it extends a base class and does not implement an interface - * so even if we create a MockPartitionReciver, it will not work as the two classes are orthogonal. + * so even if we create a MockPartitionReceiver, it will not work as the two classes are orthogonal. * * @param context - The processcontext for this processor * @param partitionId - The partition ID to retrieve a receiver by. diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java index f41f21489bde..0adf20fc5e09 100644 --- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java +++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/PutAzureEventHub.java @@ -30,7 +30,6 @@ import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; @@ -56,40 +55,22 @@ @InputRequirement(Requirement.INPUT_REQUIRED) @CapabilityDescription("Sends the contents of a FlowFile to a Windows Azure Event Hub. Note: the content of the FlowFile will be buffered into memory before being sent, " + "so care should be taken to avoid sending FlowFiles to this Processor that exceed the amount of Java Heap Space available.") -public class PutAzureEventHub extends AbstractProcessor { - static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder() - .name("Event Hub Name") - .description("The name of the Azure Event Hub to send to") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .required(true) - .build(); - static final PropertyDescriptor NAMESPACE = new PropertyDescriptor.Builder() - .name("Event Hub Namespace") - .description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to -ns") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(false) - .required(true) - .build(); +public class PutAzureEventHub extends AbstractAzureEventHub { + static final PropertyDescriptor ACCESS_POLICY = new PropertyDescriptor.Builder() - .name("Shared Access Policy Name") + .name("shared-access-send-policy-name") + .displayName("Shared Access Send Policy Name") .description("The name of the Event Hub Shared Access Policy. This Policy must have Send permissions.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(false) .required(true) .build(); - static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder() - .name("Shared Access Policy Primary Key") - .description("The primary key of the Event Hub Shared Access Policy") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(false) - .sensitive(true) - .required(true) - .build(); - + static final Relationship REL_SUCCESS = new Relationship.Builder() .name("success") .description("Any FlowFile that is successfully sent to the Azure Event Hub will be transferred to this Relationship.") .build(); + static final Relationship REL_FAILURE = new Relationship.Builder() .name("failure") .description("Any FlowFile that could not be sent to the Azure Event Hub will be transferred to this Relationship.")