From e4f8550159132d85d0e19d61f24e087e51742dee Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 23 Feb 2018 17:40:22 -0800 Subject: [PATCH 01/16] Added Pulsar processors and Controller Service --- .../nifi-pulsar-bundle/.gitignore | 5 + .../nifi-pulsar-nar/pom.xml | 48 +++ .../nifi-pulsar-processors/pom.xml | 76 ++++ .../pulsar/AbstractPulsarProcessor.java | 43 ++ .../nifi/processors/pulsar/ConsumePulsar.java | 391 ++++++++++++++++++ .../nifi/processors/pulsar/PublishPulsar.java | 372 +++++++++++++++++ .../org.apache.nifi.processor.Processor | 16 + .../pulsar/AbstractPulsarProcessorTest.java | 37 ++ .../pulsar/ConsumePulsarProcessorTest.java | 162 ++++++++ .../pulsar/MockPulsarClientService.java | 156 +++++++ .../pulsar/PublishPulsarProcessorTest.java | 206 +++++++++ nifi-nar-bundles/nifi-pulsar-bundle/pom.xml | 35 ++ .../nifi-pulsar-client-services/.gitignore | 5 + .../pom.xml | 46 +++ .../nifi-pulsar-client-service-api/pom.xml | 40 ++ .../apache/nifi/pulsar/PulsarClientPool.java | 47 +++ .../apache/nifi/pulsar/PulsarConsumer.java | 70 ++++ .../apache/nifi/pulsar/PulsarProducer.java | 64 +++ .../apache/nifi/pulsar/cache/LRUCache.java | 69 ++++ .../nifi/pulsar/pool/PoolableResource.java | 25 ++ .../pulsar/pool/PulsarConsumerFactory.java | 56 +++ .../pulsar/pool/PulsarProducerFactory.java | 53 +++ .../pool/ResourceCreationException.java | 31 ++ .../pulsar/pool/ResourceExceptionHandler.java | 23 ++ .../nifi/pulsar/pool/ResourceFactory.java | 24 ++ .../apache/nifi/pulsar/pool/ResourcePool.java | 43 ++ .../nifi/pulsar/cache/LRUCacheTest.java | 135 ++++++ .../nifi-pulsar-client-service-nar/pom.xml | 47 +++ .../nifi-pulsar-client-service/pom.xml | 64 +++ .../nifi/pulsar/StandardPulsarClientPool.java | 300 ++++++++++++++ .../pool/ResourceExceptionHandlerImpl.java | 31 ++ .../nifi/pulsar/pool/ResourcePoolImpl.java | 139 +++++++ ...g.apache.nifi.controller.ControllerService | 15 + .../org/apache/nifi/pulsar/TestProcessor.java | 45 ++ .../TestStandardPulsarClientService.java | 44 ++ .../pulsar/pool/MockPulsarClientService.java | 146 +++++++ .../nifi/pulsar/pool/ResourcePoolTest.java | 146 +++++++ .../nifi-pulsar-client-services/pom.xml | 37 ++ 38 files changed, 3292 insertions(+) create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/.gitignore create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/.gitignore create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/pom.xml diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/.gitignore b/nifi-nar-bundles/nifi-pulsar-bundle/.gitignore new file mode 100644 index 000000000000..8cfc6f727089 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/.gitignore @@ -0,0 +1,5 @@ +.project +.settings/ +*/.classpath +*/.gitignore +/target/ diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml new file mode 100644 index 000000000000..ca78a838211f --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml @@ -0,0 +1,48 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-bundle + 1.5.0 + + + nifi-pulsar-nar + nar + + true + true + + + + + org.apache.nifi + nifi-pulsar-processors + 1.5.0 + + + + org.apache.nifi + nifi-pulsar-client-service-api-nar + 1.5.0 + nar + + + + + diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml new file mode 100644 index 000000000000..5ebafd05e6fa --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml @@ -0,0 +1,76 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-bundle + 1.5.0 + + + nifi-pulsar-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-record-serialization-service-api + + + org.apache.nifi + nifi-record + + + org.apache.nifi + nifi-utils + + + org.apache.nifi + nifi-ssl-context-service-api + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.5.0 + provided + + + org.apache.pulsar + pulsar-client + 1.21.0-incubating + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java new file mode 100644 index 000000000000..a2ecf430ba0f --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.pulsar.PulsarClientPool; + +public abstract class AbstractPulsarProcessor extends AbstractProcessor { + + protected static final PropertyDescriptor PULSAR_CLIENT_SERVICE = new PropertyDescriptor.Builder() + .name("Pulsar Client Service") + .description("Specified the Pulsar Client Service that can be used to create Pulsar connections") + .required(true) + .identifiesControllerService(PulsarClientPool.class) + .build(); + + protected static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles for which all content was sent to Pulsar.") + .build(); + + protected static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("Any FlowFile that cannot be sent to Pulsar will be routed to this Relationship") + .build(); +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java new file mode 100644 index 000000000000..b84cb7aa70c5 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerConfiguration; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionType; + +@Tags({"Pulsar", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume"}) +@CapabilityDescription("Consumes messages from Apache Pulsar " + + "The complementary NiFi processor for sending messages is PublishPulsar.") +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +public class ConsumePulsar extends AbstractPulsarProcessor { + + static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); + static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); + static final AllowableValue FAILOVER = new AllowableValue("Failover", "Failover", "Multiple consumer will be able to use the same subscription name but only 1 consumer " + + "will receive the messages. If that consumer disconnects, one of the other connected consumers will start receiving messages"); + + protected static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + static final PropertyDescriptor SUBSCRIPTION = new PropertyDescriptor.Builder() + .name("Subscription") + .displayName("Subscription Name") + .description("The name of the Pulsar subscription to consume from.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + .name("Async Enabled") + .description("Control whether the messages will be consumed asyncronously or not. Messages consumed" + + " syncronously will be acknowledged immediately before processing the next message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum Async Requests") + .description("The maximum number of outstanding asynchronous consumer requests for this processor. " + + "Each asynchronous call requires memory, so avoid setting this value to high.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("50") + .build(); + + public static final PropertyDescriptor ACK_TIMEOUT = new PropertyDescriptor.Builder() + .name("Acknowledgment Timeout") + .description("Set the timeout (in milliseconds) for unacked messages, truncated to the " + + "nearest millisecond. The timeout needs to be greater than 10 seconds.") + .required(false) + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .defaultValue("10000") + .build(); + + public static final PropertyDescriptor PRIORITY_LEVEL = new PropertyDescriptor.Builder() + .name("Consumer Priority Level") + .description("Sets priority level for the shared subscription consumers to which broker " + + "gives more priority while dispatching messages. Here, broker follows descending " + + "priorities. (eg: 0=max-priority, 1, 2,..) ") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("5") + .build(); + + public static final PropertyDescriptor RECEIVER_QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Consumer receive queue size.") + .description("The consumer receive queue controls how many messages can be accumulated " + + "by the Consumer before the application calls Consumer.receive(). Using a higher " + + "value could potentially increase the consumer throughput at the expense of bigger " + + "memory utilization. \n" + + "Setting the consumer queue size as zero, \n" + + "\t - Decreases the throughput of the consumer, by disabling pre-fetching of messages. \n" + + "\t - Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer " + + "connection with broker and consumer will not be able receive any further message unless batch-message " + + "in pipeline is removed") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + public static final PropertyDescriptor SUBSCRIPTION_TYPE = new PropertyDescriptor.Builder() + .name("Subscription Type") + .description("Select the subscription type to be used when subscribing to the topic.") + .required(false) + .allowableValues(EXCLUSIVE, SHARED, FAILOVER) + .defaultValue(SHARED.getValue()) + .build(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + // Reuse the same consumer for a given topic / subscription + private PulsarConsumer consumer; + private ConsumerConfiguration consumerConfig; + + // Pool for running multiple consume Async requests + ExecutorService pool; + ExecutorCompletionService completionService; + + static { + final List properties = new ArrayList<>(); + properties.add(PULSAR_CLIENT_SERVICE); + properties.add(TOPIC); + properties.add(SUBSCRIPTION); + properties.add(ASYNC_ENABLED); + properties.add(MAX_ASYNC_REQUESTS); + properties.add(ACK_TIMEOUT); + properties.add(PRIORITY_LEVEL); + properties.add(RECEIVER_QUEUE_SIZE); + properties.add(SUBSCRIPTION_TYPE); + + PROPERTIES = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + RELATIONSHIPS = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTIES; + } + + @OnScheduled + public void init(ProcessContext context) { + pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); + completionService = new ExecutorCompletionService<>(pool); + } + + @OnUnscheduled + public void shutDown() { + // Stop all the async consumers + pool.shutdownNow(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + try { + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + // Launch consumers + consumeAsync(context, session); + + // Handle completed consumers + handleAsync(context, session); + + } else { + consume(context, session); + } + } catch (PulsarClientException e) { + getLogger().error("Unable to consume from Pulsar Topic ", e); + context.yield(); + throw new ProcessException(e); + } + + } + + private void handleAsync(ProcessContext context, ProcessSession session) { + + try { + Future done = completionService.take(); + Message msg = done.get(); + + if (msg != null) { + FlowFile flowFile = null; + final byte[] value = msg.getData(); + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + } + + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.transfer(flowFile, REL_SUCCESS); + session.commit(); + getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); + } + + } catch (InterruptedException | ExecutionException | PulsarClientException e) { + getLogger().error("Trouble consuming messages ", e); + } + + } + + @OnStopped + public void close(final ProcessContext context) { + + getLogger().info("Disconnecting Pulsar Consumer"); + if (consumer != null) { + + context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getConsumerPool().evict(consumer); + } + + consumer = null; + } + + /* + * For now let's assume that this processor will be configured to run for a longer + * duration than 0 milliseconds. So we will be grabbing as many messages off the topic + * as possible and committing them as FlowFiles + */ + private void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException { + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + completionService.submit(new Callable() { + @Override + public Message call() throws Exception { + return consumer.receiveAsync().get(); + } + }); + + } + + /* + * When this Processor expects to receive many small files, it may + * be advisable to create several FlowFiles from a single session + * before committing the session. Typically, this allows the Framework + * to treat the content of the newly created FlowFiles much more efficiently. + */ + private void consume(ProcessContext context, ProcessSession session) throws PulsarClientException { + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + final ComponentLog logger = getLogger(); + final Message msg; + FlowFile flowFile = null; + + try { + + msg = consumer.receive(); + final byte[] value = msg.getData(); + + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.transfer(flowFile, REL_SUCCESS); + logger.info("Created {} from {} messages received from Pulsar Server and transferred to 'success'", + new Object[]{flowFile, 1}); + + session.commit(); + + /* + * This Processor acknowledges receipt of the data and/or removes the data + * from the external source in order to prevent receipt of duplicate files. + * This is done only after the ProcessSession by which the FlowFile was created + * has been committed! Failure to adhere to this principle may result in data + * loss, as restarting NiFi before the session has been committed will result + * in the temporary file being deleted. Note, however, that it is possible using + * this approach to receive duplicate data because the application could be + * restarted after committing the session and before acknowledging or removing + * the data from the external source. In general, though, potential data duplication + * is preferred over potential data loss. + */ + getLogger().info("Acknowledging message " + msg.getMessageId()); + consumer.acknowledge(msg); + + } else { + // We didn't consume any data, so + session.commit(); + } + + } catch (PulsarClientException e) { + context.yield(); + session.rollback(); + } + + } + + private PulsarConsumer getWrappedConsumer(ProcessContext context) throws PulsarClientException { + + if (consumer != null) + return consumer; + + final PulsarClientPool pulsarClientService = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class); + + try { + consumer = pulsarClientService.getConsumerPool() + .acquire(getConsumerProperties(context)); + + if (consumer == null || consumer.getConsumer() == null) { + throw new PulsarClientException("Unable to create Pulsar Consumer"); + } + + return consumer; + } catch (final InterruptedException ex) { + return null; + } + } + + private Properties getConsumerProperties(ProcessContext context) { + + Properties props = new Properties(); + props.put(PulsarConsumerFactory.TOPIC_NAME, context.getProperty(TOPIC).getValue()); + props.put(PulsarConsumerFactory.SUBSCRIPTION_NAME, context.getProperty(SUBSCRIPTION).getValue()); + props.put(PulsarConsumerFactory.CONSUMER_CONFIG, getConsumerConfig(context)); + return props; + } + + private ConsumerConfiguration getConsumerConfig(ProcessContext context) { + + if (consumerConfig == null) { + consumerConfig = new ConsumerConfiguration(); + + if (context.getProperty(ACK_TIMEOUT).isSet()) + consumerConfig.setAckTimeout(context.getProperty(ACK_TIMEOUT).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(PRIORITY_LEVEL).isSet()) + consumerConfig.setPriorityLevel(context.getProperty(PRIORITY_LEVEL).asInteger()); + + if (context.getProperty(RECEIVER_QUEUE_SIZE).isSet()) + consumerConfig.setReceiverQueueSize(context.getProperty(RECEIVER_QUEUE_SIZE).asInteger()); + + if (context.getProperty(SUBSCRIPTION_TYPE).isSet()) + consumerConfig.setSubscriptionType(SubscriptionType.valueOf(context.getProperty(SUBSCRIPTION_TYPE).getValue())); + } + + return consumerConfig; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java new file mode 100644 index 000000000000..20b93ddfd234 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.cache.LRUCache; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.util.StringUtils; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConfiguration; +import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode; +import org.apache.pulsar.client.api.PulsarClientException; + +@Tags({"Apache", "Pulsar", "Put", "Send", "Message", "PubSub"}) +@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Pulsar using the Pulsar 1.21 Producer API." + + "The messages to send may be individual FlowFiles or may be delimited, using a " + + "user-specified delimiter, such as a new-line. " + + "The complementary NiFi processor for fetching messages is ConsumePulsar.") +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " + + "FlowFiles that are routed to success.") +public class PublishPulsar extends AbstractPulsarProcessor { + + protected static final String MSG_COUNT = "msg.count"; + + static final AllowableValue COMPRESSION_TYPE_NONE = new AllowableValue("NONE", "None", "No compression"); + static final AllowableValue COMPRESSION_TYPE_LZ4 = new AllowableValue("LZ4", "LZ4", "Compress with LZ4 algorithm."); + static final AllowableValue COMPRESSION_TYPE_ZLIB = new AllowableValue("ZLIB", "ZLIB", "Compress with ZLib algorithm"); + + static final AllowableValue MESSAGE_ROUTING_MODE_CUSTOM_PARTITION = new AllowableValue("CustomPartition", "Custom Partition", "Route messages to a custom partition"); + static final AllowableValue MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION = new AllowableValue("RoundRobinPartition", "Round Robin Partition", "Route messages to all partitions in a round robin manner"); + static final AllowableValue MESSAGE_ROUTING_MODE_SINGLE_PARTITION = new AllowableValue("SinglePartition", "Single Partition", "Route messages to a single partition"); + + public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + .name("Async Enabled") + .description("Control whether the messages will be sent asyncronously or not. Messages sent" + + " syncronously will be acknowledged immediately before processing the next message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor BATCHING_ENABLED = new PropertyDescriptor.Builder() + .name("Batching Enabled") + .description("Control whether automatic batching of messages is enabled for the producer. " + + "default: false [No batching] When batching is enabled, multiple calls to " + + "Producer.sendAsync can result in a single batch to be sent to the broker, leading " + + "to better throughput, especially when publishing small messages. If compression is " + + "enabled, messages will be compressed at the batch level, leading to a much better " + + "compression ratio for similar headers or contents. When enabled default batch delay " + + "is set to 10 ms and default batch size is 1000 messages") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor BATCHING_MAX_MESSAGES = new PropertyDescriptor.Builder() + .name("Batching Max Messages") + .description("Set the maximum number of messages permitted in a batch. default: " + + "1000 If set to a value greater than 1, messages will be queued until this " + + "threshold is reached or batch interval has elapsed") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + public static final PropertyDescriptor BATCH_INTERVAL = new PropertyDescriptor.Builder() + .name("Batch Interval") + .description("Set the time period within which the messages sent will be batched default: 10ms " + + "if batch messages are enabled. If set to a non zero value, messages will be queued until " + + "this time interval or until the Batching Max Messages threshould has been reached") + .required(false) + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .defaultValue("10") + .build(); + + public static final PropertyDescriptor BLOCK_IF_QUEUE_FULL = new PropertyDescriptor.Builder() + .name("Block if Message Queue Full") + .description("Set whether the processor should block when the outgoing message queue is full. " + + "Default is false. If set to false, send operations will immediately fail with " + + "ProducerQueueIsFullError when there is no space left in pending queue.") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() + .name("Compression Type") + .description("Set the compression type for the producer.") + .required(false) + .allowableValues(COMPRESSION_TYPE_NONE, COMPRESSION_TYPE_LZ4, COMPRESSION_TYPE_ZLIB) + .defaultValue(COMPRESSION_TYPE_NONE.getValue()) + .build(); + + public static final PropertyDescriptor MESSAGE_ROUTING_MODE = new PropertyDescriptor.Builder() + .name("Message Routing Mode") + .description("Set the message routing mode for the producer. This applies only if the destination topic is partitioned") + .required(false) + .allowableValues(MESSAGE_ROUTING_MODE_CUSTOM_PARTITION, MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION, MESSAGE_ROUTING_MODE_SINGLE_PARTITION) + .defaultValue(MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION.getValue()) + .build(); + + public static final PropertyDescriptor PENDING_MAX_MESSAGES = new PropertyDescriptor.Builder() + .name("Max Pending Messages") + .description("Set the max size of the queue holding the messages pending to receive an " + + "acknowledgment from the broker.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + private LRUCache producers; + private ProducerConfiguration producerConfig; + + static { + final List properties = new ArrayList<>(); + properties.add(PULSAR_CLIENT_SERVICE); + properties.add(TOPIC); + properties.add(ASYNC_ENABLED); + properties.add(BATCHING_ENABLED); + properties.add(BATCHING_MAX_MESSAGES); + properties.add(BATCH_INTERVAL); + properties.add(BLOCK_IF_QUEUE_FULL); + properties.add(COMPRESSION_TYPE); + properties.add(MESSAGE_ROUTING_MODE); + properties.add(PENDING_MAX_MESSAGES); + + PROPERTIES = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + RELATIONSHIPS = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTIES; + } + + @OnStopped + public void cleanUp(final ProcessContext context) { + // Close all of the producers and invalidate them, so they get removed from the Resource Pool + getProducerCache(context).clear(); + } + + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + FlowFile flowFile = session.get(); + + if (flowFile == null) + return; + + final ComponentLog logger = getLogger(); + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + + if (StringUtils.isBlank(topic)) { + logger.error("Invalid topic specified {}", new Object[] {topic}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + // Read the contents of the FlowFile into a byte array + final byte[] messageContent = new byte[(int) flowFile.getSize()]; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + StreamUtils.fillBuffer(in, messageContent, true); + } + }); + + // Nothing to do, so skip this Flow file. + if (messageContent == null || messageContent.length < 1) { + session.transfer(flowFile, REL_SUCCESS); + return; + } + + try { + + Producer producer = getWrappedProducer(topic, context).getProducer(); + + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + this.sendAsync(producer, session, flowFile, messageContent); + } else { + this.send(producer, session, flowFile, messageContent); + } + + } catch (final PulsarClientException e) { + logger.error("Failed to connect to Pulsar Server due to {}", new Object[]{e}); + session.penalize(flowFile); + session.transfer(flowFile, REL_FAILURE); + } + + } + + + private void send(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) throws PulsarClientException { + + MessageId msgId = producer.send(messageContent); + + if (msgId != null) { + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } else { + session.transfer(flowFile, REL_FAILURE); + } + + } + + private void sendAsync(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) { + + producer.sendAsync(messageContent).handle((msgId, ex) -> { + if (msgId != null) { + return msgId; + } else { + // TODO Communicate the error back up to the onTrigger method so we can invalidate this producer. + getLogger().warn("Problem ", ex); + return null; + } + }); + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } + + private PulsarProducer getWrappedProducer(String topic, ProcessContext context) throws PulsarClientException, IllegalArgumentException { + + PulsarProducer producer = getProducerCache(context).get(topic); + + if (producer != null) + return producer; + + try { + producer = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool().acquire(getProducerProperties(context, topic)); + + if (producer != null) { + producers.put(topic, producer); + } + + return producer; + + } catch (InterruptedException e) { + return null; + } + + } + + private LRUCache getProducerCache(ProcessContext context) { + if (producers == null) { + + ResourcePool pool = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool(); + + producers = new LRUCache (20, pool); + } + + return producers; + } + + private Properties getProducerProperties(ProcessContext context, String topic) { + + Properties props = new Properties(); + props.put(PulsarProducerFactory.TOPIC_NAME, topic); + props.put(PulsarProducerFactory.PRODUCER_CONFIG, getProducerConfig(context)); + return props; + } + + private ProducerConfiguration getProducerConfig(ProcessContext context) { + + if (producerConfig == null) { + producerConfig = new ProducerConfiguration(); + + if (context.getProperty(BATCHING_ENABLED).isSet()) + producerConfig.setBatchingEnabled(context.getProperty(BATCHING_ENABLED).asBoolean()); + + if (context.getProperty(BATCHING_MAX_MESSAGES).isSet()) + producerConfig.setBatchingMaxMessages(context.getProperty(BATCHING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(BATCH_INTERVAL).isSet()) + producerConfig.setBatchingMaxPublishDelay(context.getProperty(BATCH_INTERVAL).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(BLOCK_IF_QUEUE_FULL).isSet()) + producerConfig.setBlockIfQueueFull(context.getProperty(BLOCK_IF_QUEUE_FULL).asBoolean()); + + if (context.getProperty(COMPRESSION_TYPE).isSet()) + producerConfig.setCompressionType(CompressionType.valueOf(context.getProperty(COMPRESSION_TYPE).getValue())); + + if (context.getProperty(PENDING_MAX_MESSAGES).isSet()) + producerConfig.setMaxPendingMessages(context.getProperty(PENDING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(MESSAGE_ROUTING_MODE).isSet()) + producerConfig.setMessageRoutingMode(MessageRoutingMode.valueOf(context.getProperty(MESSAGE_ROUTING_MODE).getValue())); + } + + return producerConfig; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 000000000000..93e412f3fc13 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# 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. +org.apache.nifi.processors.pulsar.ConsumePulsar +org.apache.nifi.processors.pulsar.PublishPulsar \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java new file mode 100644 index 000000000000..c11008209f63 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.TestRunner; +import org.apache.pulsar.client.api.PulsarClient; +import org.mockito.Mock; + +public abstract class AbstractPulsarProcessorTest { + + protected TestRunner runner; + + @Mock + protected PulsarClient mockClient; + + protected void addPulsarClientService() throws InitializationException { + final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); + runner.addControllerService("pulsarClient", pulsarClient); + runner.enableControllerService(pulsarClient); + runner.setProperty(PublishPulsar.PULSAR_CLIENT_SERVICE, "pulsarClient"); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java new file mode 100644 index 000000000000..64550d442043 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunners; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.*; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { + + @Mock + Consumer mockConsumer; + + @Mock + Message mockMessage; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @Before + public void init() throws InitializationException { + runner = TestRunners.newTestRunner(ConsumePulsar.class); + + mockClient = mock(PulsarClient.class); + mockConsumer = mock(Consumer.class); + mockMessage = mock(Message.class); + + try { + when(mockClient.subscribe(anyString(), anyString())).thenReturn(mockConsumer); + when(mockConsumer.receive()).thenReturn(mockMessage); + + CompletableFuture future = CompletableFuture.supplyAsync(() -> { + return mockMessage; + }); + + when(mockConsumer.receiveAsync()).thenReturn(future); + + } catch (PulsarClientException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + + addPulsarClientService(); + } + + @Test + public void emptyMessageTest() { + when(mockMessage.getData()).thenReturn("".getBytes()); + + runner.setProperty(ConsumePulsar.TOPIC, "foo"); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.run(); + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + } + + @Test + public void singleSyncMessageTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", false, 1); + } + + @Test + public void multipleSyncMessagesTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", false, 40); + } + + @Test + public void singleAsyncMessageTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", true, 1); + } + + @Test + public void multipleAsyncMessagesTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", true, 40); + } + + /* + * Verify that the consumer gets closed. + */ + @Test + public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { + when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); + + runner.setProperty(ConsumePulsar.TOPIC, "foo"); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.run(10, true); + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + + runner.assertQueueEmpty(); + + // Verify that the receive method on the consumer was called 10 times + verify(mockConsumer, times(10)).receive(); + + // Verify that each message was acknowledged + verify(mockConsumer, times(10)).acknowledge(mockMessage); + + // Verify that the consumer was closed + verify(mockConsumer, times(1)).close(); + + } + + private void sendMessages(String msg, String topic, String sub, boolean async, int itertions) throws PulsarClientException { + + when(mockMessage.getData()).thenReturn(msg.getBytes()); + + runner.setProperty(ConsumePulsar.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar.TOPIC, topic); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, sub); + runner.run(itertions, true); + + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar.REL_SUCCESS); + assertEquals(itertions, flowFiles.size()); + + for (MockFlowFile ff : flowFiles) { + ff.assertContentEquals(msg); + } + + if (async) { + verify(mockConsumer, times(itertions)).receiveAsync(); + } else { + verify(mockConsumer, times(itertions)).receive(); + } + + // Verify that every message was acknowledged + if (async) { + verify(mockConsumer, times(itertions)).acknowledgeAsync(mockMessage); + } else { + verify(mockConsumer, times(itertions)).acknowledge(mockMessage); + } + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java new file mode 100644 index 000000000000..3f7829272eb0 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Rule; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.stubbing.Answer; + +import static org.mockito.Mockito.*; + +import java.util.Properties; + +public class MockPulsarClientService extends AbstractControllerService implements PulsarClientPool { + + @Mock + PulsarClient mockClient; + + @Mock + ResourcePool mockProducerPool; + + @Mock + ResourcePool mockConsumerPool; + + @Mock + Producer mockProducer; + + @Mock + Consumer mockConsumer; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @SuppressWarnings({ "unchecked", "rawtypes" }) + public MockPulsarClientService(PulsarClient mockClient2) { + this.mockClient = mockClient2; + mockProducerPool = mock(ResourcePool.class); + mockConsumerPool = mock(ResourcePool.class); + mockProducer = mock(Producer.class); + mockConsumer = mock(Consumer.class); + + try { + when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarProducer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getProducer(props); + } + } + ); + + when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarConsumer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getConsumer(props); + } + } + ); + + + doAnswer(new Answer() { + public Object answer(InvocationOnMock invocation){ + PulsarConsumer consumer = invocation.getArgumentAt(0, PulsarConsumer.class); + consumer.close(); + return null; + } + }).when(mockConsumerPool).evict(any(PulsarConsumer.class)); + + } catch (InterruptedException ex) { + + } + + try { + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(null); + } catch (PulsarClientException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + + + public Producer getMockProducer() { + return mockProducer; + } + + public PulsarClient getMockClient() { + return mockClient; + } + + public PulsarProducer getProducer(Properties props) { + String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); + try { + return new PulsarProducer(mockClient.createProducer(topic), topic); + } catch (PulsarClientException e) { + return null; + } + } + + public PulsarConsumer getConsumer(Properties props) { + String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); + String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); + try { + return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + } catch (PulsarClientException e) { + return null; + } + } + + @Override + public ResourcePool getProducerPool() { + return mockProducerPool; + } + + + @Override + public ResourcePool getConsumerPool() { + return mockConsumerPool; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java new file mode 100644 index 000000000000..30717a649e68 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.io.UnsupportedEncodingException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunners; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; +import org.mockito.Mock; + +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.*; + +public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { + + @Mock + Producer mockProducer; + + @Before + public void init() throws InitializationException { + runner = TestRunners.newTestRunner(PublishPulsar.class); + + mockClient = mock(PulsarClient.class); + mockProducer = mock(Producer.class); + + try { + // Use the mockProducer for all Producer interactions + when(mockClient.createProducer(anyString())).thenReturn(mockProducer); + + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(mock(MessageId.class)); + + CompletableFuture future = CompletableFuture.supplyAsync(() -> { + return mock(MessageId.class); + }); + + when(mockProducer.sendAsync(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(future); + + + } catch (PulsarClientException e) { + e.printStackTrace(); + } + + addPulsarClientService(); + } + + @Test + public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap (); + attributes.put(PublishPulsar.TOPIC.getName(), ""); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_FAILURE); + + // Confirm that no Producer as created + verify(mockClient, times(0)).createProducer(anyString()); + } + + @Test + public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap (); + attributes.put(PublishPulsar.TOPIC.getName(), "topic-b"); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + // Verify that we sent the data to topic-b. + verify(mockClient, times(1)).createProducer("topic-b"); + } + + @Test + public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + + final String content = "some content"; + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + outFile.assertContentEquals(content); + + // Verify that we sent the data to my-topic. + verify(mockClient, times(1)).createProducer("my-topic"); + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(1)).send(content.getBytes()); + } + + @Test + public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + + final String content = "some content"; + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + outFile.assertContentEquals(content); + + // Verify that we sent the data to my-topic. + verify(mockClient, times(1)).createProducer("my-topic"); + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(1)).sendAsync(content.getBytes()); + } + + @Test + public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + final String content = "some content"; + + // Hack, since runner.run(20, false); doesn't work as advertised + for (int idx = 0; idx < 20; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + } + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(20)).send(content.getBytes()); + } + + @Test + public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + final String content = "some content"; + + // Hack, since runner.run(20, false); doesn't work as advertised + for (int idx = 0; idx < 20; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + } + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(20)).sendAsync(content.getBytes()); + } + + @Test + public void stressTest() throws UnsupportedEncodingException { + runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + final String content = "some content"; + + for (int idx = 0; idx < 9999; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml new file mode 100644 index 000000000000..ccb90f23350b --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml @@ -0,0 +1,35 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 1.5.0 + + + org.apache.nifi + nifi-pulsar-bundle + 1.5.0 + pom + + + nifi-pulsar-processors + nifi-pulsar-nar + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore b/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore new file mode 100644 index 000000000000..8cfc6f727089 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore @@ -0,0 +1,5 @@ +.project +.settings/ +*/.classpath +*/.gitignore +/target/ diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml new file mode 100644 index 000000000000..a0bd2122c2ae --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml @@ -0,0 +1,46 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service-api-nar + 1.5.0 + nar + + true + true + + + + + org.apache.nifi + nifi-standard-services-api-nar + nar + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.5.0 + + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml new file mode 100644 index 000000000000..2218c3258649 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service-api + jar + + + + org.apache.nifi + nifi-api + provided + + + org.apache.pulsar + pulsar-client + 1.21.0-incubating + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java new file mode 100644 index 000000000000..be57a42a0d8c --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java @@ -0,0 +1,47 @@ +/* + * 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.pulsar; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.pulsar.pool.ResourcePool; + + +@Tags({"Pulsar"}) +@CapabilityDescription("Provides the ability to create Pulsar Producer / Consumer instances on demand, based on the configuration." + + "properties defined") +public interface PulsarClientPool extends ControllerService { + + /* + public PulsarProducer getProducer(Properties properties) throws PulsarClientException; + + public void release(PulsarProducer producer); + + public void invalidate(PulsarProducer producer); + + public PulsarConsumer getConsumer(Properties properties) throws PulsarClientException; + + public void release(PulsarConsumer consumer); + + public void invalidate(PulsarConsumer consumer); +*/ + + public ResourcePool getProducerPool(); + + public ResourcePool getConsumerPool(); +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java new file mode 100644 index 000000000000..59c98cdd0246 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java @@ -0,0 +1,70 @@ +/* + * 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.pulsar; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.ConsumerStats; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("unused") +public class PulsarConsumer implements PoolableResource { + + private static final Logger logger = LoggerFactory.getLogger(PulsarConsumer.class); + + private final Consumer consumer; + private final String topicName; + private final String subscriptionName; + private boolean closed = false; + + public PulsarConsumer(Consumer consumer, String topic, String subscription) throws PulsarClientException { + this.consumer = consumer; + this.topicName = topic; + this.subscriptionName = subscription; + } + + public void close() { + + logger.info("Closing consumer for topic {} and subscription {}", new Object[] {topicName, subscriptionName}); + closed = true; + + try { + consumer.unsubscribe(); + consumer.close(); + } catch (PulsarClientException e) { + logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); + closed = false; + } + } + + public boolean isClosed() { + return closed; + } + + public Consumer getConsumer() { + return this.consumer; + } + + public ConsumerStats getStats() { + return this.consumer.getStats(); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java new file mode 100644 index 000000000000..a13975b270c2 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java @@ -0,0 +1,64 @@ +/* + * 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.pulsar; + +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PulsarProducer implements PoolableResource { + + private static final Logger logger = LoggerFactory.getLogger(PulsarProducer.class); + + private final Producer producer; + private boolean closed = false; + private final String topicName; + + public PulsarProducer(Producer producer, String topicName) throws PulsarClientException { + this.topicName = topicName; + this.producer = producer; + } + + public Producer getProducer() { + return producer; + } + + public String getName() { + return topicName; + } + + public boolean isClosed() { + return this.closed; + } + + public void close() { + + logger.info("Closing producer for topic {} ", new Object[] {topicName}); + + this.closed = true; + try { + producer.close(); + } catch (PulsarClientException e) { + logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); + closed = false; + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java new file mode 100644 index 000000000000..86a0fc2afab0 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java @@ -0,0 +1,69 @@ +/* + * 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.pulsar.cache; + +import java.util.LinkedHashMap; + +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.nifi.pulsar.pool.ResourcePool; + +public class LRUCache { + + private LinkedHashMap lruCacheMap; + private final int capacity; + private final boolean SORT_BY_ACCESS = true; + private final float LOAD_FACTOR = 0.75F; + private final ResourcePool resourcePool; + + public LRUCache(int capacity, ResourcePool resourcePool){ + this.capacity = capacity; + this.lruCacheMap = new LinkedHashMap<>(capacity, LOAD_FACTOR, SORT_BY_ACCESS); + this.resourcePool = resourcePool; + } + + public V get(K k){ + return lruCacheMap.get(k); + } + + public void put(K k, V v){ + if(lruCacheMap.containsKey(k)){ + lruCacheMap.remove(k); + } else if(lruCacheMap.size() >= capacity){ + K victimKey = lruCacheMap.keySet().iterator().next(); + V victim = lruCacheMap.get(victimKey); + lruCacheMap.remove(victimKey); + resourcePool.evict(victim); + } + lruCacheMap.put(k, v); + } + + public int getSize() { + return lruCacheMap.size(); + } + + public void clear() { + for (V victim :lruCacheMap.values()) { + resourcePool.evict(victim); + } + + lruCacheMap.clear(); + } + + public void printSequence(){ + System.out.println(lruCacheMap.keySet()); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java new file mode 100644 index 000000000000..6af5845e0885 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java @@ -0,0 +1,25 @@ +/* + * 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.pulsar.pool; + +public interface PoolableResource { + + public void close(); + + public boolean isClosed(); + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java new file mode 100644 index 000000000000..6794d2a7165e --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java @@ -0,0 +1,56 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.pulsar.client.api.ConsumerConfiguration; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; + +public class PulsarConsumerFactory implements ResourceFactory { + + public static final String TOPIC_NAME = "topic"; + public static final String SUBSCRIPTION_NAME = "subscription"; + public static final String CONSUMER_CONFIG = "Consumer-Configuration"; + + private PulsarClient client; + + public PulsarConsumerFactory(PulsarClient client) { + this.client = client; + } + + @Override + public PulsarConsumer create(Properties props) throws ResourceCreationException { + + String topic = props.getProperty(TOPIC_NAME); + String subscription = props.getProperty(SUBSCRIPTION_NAME); + ConsumerConfiguration config = (ConsumerConfiguration) props.get(CONSUMER_CONFIG); + + try { + // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice + return (config == null) ? new PulsarConsumer(client.subscribe(topic, subscription), topic, subscription) : + new PulsarConsumer(client.subscribe(topic, subscription, config), topic, subscription); + + } catch (PulsarClientException e) { + throw new ResourceCreationException(e); + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java new file mode 100644 index 000000000000..37165fc45ee6 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java @@ -0,0 +1,53 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.pulsar.client.api.ProducerConfiguration; +import org.apache.pulsar.client.api.PulsarClient; + +public class PulsarProducerFactory implements ResourceFactory { + + public static final String TOPIC_NAME = "topic"; + public static final String PRODUCER_CONFIG = "Producer-Configuration"; + + private PulsarClient client; + + public PulsarProducerFactory(PulsarClient client) { + this.client = client; + } + + @Override + public PulsarProducer create(Properties props) throws ResourceCreationException { + + String topic = props.getProperty(TOPIC_NAME); + ProducerConfiguration config = (ProducerConfiguration) props.get(PRODUCER_CONFIG); + + try { + // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice + return (config == null) ? new PulsarProducer(client.createProducer(topic), topic) : + new PulsarProducer(client.createProducer(topic, config), topic); + + } catch (Exception e) { + throw new ResourceCreationException(e); + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java new file mode 100644 index 000000000000..7eb3ea6ccee5 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java @@ -0,0 +1,31 @@ +/* + * 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.pulsar.pool; + +public class ResourceCreationException extends Exception { + + private static final long serialVersionUID = -1796731800483756134L; + + public ResourceCreationException(String s) { + super(s); + } + + public ResourceCreationException(Exception e) { + super(e); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java new file mode 100644 index 000000000000..403d95cbcd71 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java @@ -0,0 +1,23 @@ + +/* + * 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.pulsar.pool; + +public interface ResourceExceptionHandler { + + void handle(Exception exc); +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java new file mode 100644 index 000000000000..1cf0500d441d --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.pulsar.pool; + +import java.util.Properties; + +public interface ResourceFactory { + + public R create(Properties props) throws ResourceCreationException; +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java new file mode 100644 index 000000000000..e67b8f4b58e7 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java @@ -0,0 +1,43 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +public interface ResourcePool { + + /** + * Acquire a resource from the pool. Creating one if necessary + * @param props + * @return + * @throws InterruptedException + */ + public R acquire(Properties props) throws InterruptedException; + + /** + * Evict the resource from the pool, destroying it. + * Call this method is the resource is known to be in an unusable state. + * @param resource + */ + public void evict(R resource); + + /** + * Place the resource back into the pool for future use. + * @param resource + */ + public void release(R resource); +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java new file mode 100644 index 000000000000..53f3991b116d --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java @@ -0,0 +1,135 @@ +/* + * 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.pulsar.cache; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +public class LRUCacheTest { + + @Mock + private ResourcePool mockedResourcePool; + + @Mock + private PulsarProducer mockedPulsarProducer; + + @SuppressWarnings("unchecked") + @Before + public void setUp() throws InterruptedException { + mockedResourcePool = mock(ResourcePool.class); + mockedPulsarProducer = mock(PulsarProducer.class); + + when(mockedResourcePool.acquire(any(Properties.class))).thenReturn(mockedPulsarProducer); + } + + /** + * Make sure the LRUCache functions as a Map + */ + @Test + public void simpleTest() { + LRUCache cache = new LRUCache (10, mockedResourcePool); + + for(Character i='A'; i<='E'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + assertEquals(5, cache.getSize()); + + for(Character i='A'; i<='E'; i++){ + assertNotNull( cache.get(i.toString())); + } + } + + @Test + public void evictionTest() { + + LRUCache cache = new LRUCache (5, mockedResourcePool); + + for(Character i='A'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + // Make sure we only have 5 items in the cache + assertEquals(5, cache.getSize()); + + // Make sure we have the last 5 items added to the cache + for(Character i='V'; i<='Z'; i++){ + assertNotNull( cache.get(i.toString())); + } + + // Make sure the evict method on the resource pool was called 21 times, once for every item removed. + verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); + } + + @Test + public void evictionLruTest() { + + LRUCache cache = new LRUCache (5, mockedResourcePool); + + final Character A = 'A'; + + // Write 25 items to the cache, and the letter 'A' every other put. + for(Character i='B'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + cache.put(A.toString(), mockedPulsarProducer); + } + + // Make sure we only have 5 items in the cache + assertEquals(5, cache.getSize()); + + // Make sure that the letter 'A' is still in the cache due to frequent access + assertNotNull( cache.get(A.toString()) ); + + // Make sure we have the last 4 items added to the cache + for(Character i='W'; i<='Z'; i++){ + assertNotNull( cache.get(i.toString())); + } + + // Make sure the evict method on the resource pool was called 21 times, once for every item removed. + verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); + } + + @Test + public void clearTest() { + LRUCache cache = new LRUCache (26, mockedResourcePool); + + for(Character i='A'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + // Make sure we only have all the items in the cache + assertEquals(26, cache.getSize()); + + cache.clear(); + + // Make sure all the items were removed + assertEquals(0, cache.getSize()); + + // Make sure all the items were evicted from the underlying resource pool + verify(mockedResourcePool, times(26)).evict(mockedPulsarProducer); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml new file mode 100644 index 000000000000..cb3c116e3772 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml @@ -0,0 +1,47 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service-nar + 1.5.0 + nar + + true + true + + + + + org.apache.nifi + nifi-pulsar-client-service-api-nar + 1.5.0 + nar + + + org.apache.nifi + nifi-pulsar-client-service + 1.5.0 + + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml new file mode 100644 index 000000000000..366b9e66d4b8 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml @@ -0,0 +1,64 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service + jar + + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.5.0 + + + org.apache.nifi + nifi-api + provided + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-ssl-context-service-api + provided + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java new file mode 100644 index 000000000000..e965d99b89f0 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java @@ -0,0 +1,300 @@ +/* + * 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.pulsar; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.nifi.pulsar.pool.ResourcePoolImpl; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.pulsar.client.api.ClientConfiguration; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; + +@Tags({ "Pulsar"}) +@CapabilityDescription("Standard ControllerService implementation of PulsarClientService.") +public class StandardPulsarClientPool extends AbstractControllerService implements PulsarClientPool { + + public static final PropertyDescriptor PULSAR_SERVICE_URL = new PropertyDescriptor + .Builder().name("PULSAR_SERVICE_URL") + .displayName("Pulsar Service URL") + .description("URL for the Pulsar cluster, e.g localhost:6650") + .required(true) + .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR) + .build(); + + public static final PropertyDescriptor CONCURRENT_LOOKUP_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum concurrent lookup-requests") + .description("Number of concurrent lookup-requests allowed on each broker-connection to prevent " + + "overload on broker. (default: 5000) It should be configured with higher value only in case " + + "of it requires to produce/subscribe on thousands of topics") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("5000") + .build(); + + public static final PropertyDescriptor CONNECTIONS_PER_BROKER = new PropertyDescriptor.Builder() + .name("Maximum connects per Pulsar broker") + .description("Sets the max number of connection that the client library will open to a single broker.\n" + + "By default, the connection pool will use a single connection for all the producers and consumers. " + + "Increasing this parameter may improve throughput when using many producers over a high latency connection") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor IO_THREADS = new PropertyDescriptor.Builder() + .name("I/O Threads") + .description("The number of threads to be used for handling connections to brokers (default: 1 thread)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor LISTENER_THREADS = new PropertyDescriptor.Builder() + .name("Listener Threads") + .description("The number of threads to be used for message listeners (default: 1 thread)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAXIMUM_REJECTED_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum rejected requests per connection") + .description("Max number of broker-rejected requests in a certain time-frame (30 seconds) after " + + "which current connection will be closed and client creates a new connection that give " + + "chance to connect a different broker (default: 50)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("50") + .build(); + + public static final PropertyDescriptor OPERATION_TIMEOUT = new PropertyDescriptor.Builder() + .name("Operation Timeout") + .description("Producer-create, subscribe and unsubscribe operations will be retried until this " + + "interval, after which the operation will be maked as failed (default: 30 seconds)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("30") + .build(); + + public static final PropertyDescriptor STATS_INTERVAL = new PropertyDescriptor.Builder() + .name("Stats interval") + .description("The interval between each stat info (default: 60 seconds) Stats will be activated " + + "with positive statsIntervalSeconds It should be set to at least 1 second") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("60") + .build(); + + public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder() + .name("Use TCP nodelay flag") + .description("Configure whether to use TCP no-delay flag on the connection, to disable Nagle algorithm.\n" + + "No-delay features make sure packets are sent out on the network as soon as possible, and it's critical " + + "to achieve low latency publishes. On the other hand, sending out a huge number of small packets might " + + "limit the overall throughput, so if latency is not a concern, it's advisable to set the useTcpNoDelay " + + "flag to false.") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor MAX_PRODUCERS = new PropertyDescriptor + .Builder().name("MAX_PRODUCERS") + .displayName("Producer Pool Size") + .description("The Maximum Number of Pulsar Producers created by this Pulsar Client Pool") + .required(true) + .defaultValue("10") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final PropertyDescriptor MAX_CONSUMERS = new PropertyDescriptor + .Builder().name("MAX_CONSUMERS") + .displayName("Consumer Pool Size") + .description("The Maximum Number of Pulsar consumers created by this Pulsar Client Pool") + .required(true) + .defaultValue("10") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("ssl.context.service") + .displayName("SSL Context Service") + .description("Specifies the SSL Context Service to use for communicating with Pulsar.") + .required(false) + .identifiesControllerService(SSLContextService.class) + .build(); + + private static final List properties; + private volatile PulsarClient client; + + private volatile ResourcePoolImpl producers; + private volatile ResourcePoolImpl consumers; + private ClientConfiguration clientConfig; + + static { + final List props = new ArrayList<>(); + props.add(PULSAR_SERVICE_URL); + props.add(MAX_CONSUMERS); + props.add(MAX_PRODUCERS); + props.add(CONCURRENT_LOOKUP_REQUESTS); + props.add(CONNECTIONS_PER_BROKER); + props.add(IO_THREADS); + props.add(LISTENER_THREADS); + props.add(MAXIMUM_REJECTED_REQUESTS); + props.add(OPERATION_TIMEOUT); + props.add(STATS_INTERVAL); + props.add(USE_TCP_NO_DELAY); + properties = Collections.unmodifiableList(props); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + /** + * @param context + * the configuration context + * @throws InitializationException + * if unable to create a database connection + */ + @OnEnabled + public void onEnabled(final ConfigurationContext context) throws InitializationException { + + createClient(context); + + if (this.client == null) + throw new InitializationException("Unable to create Pulsar Client"); + + producers = new ResourcePoolImpl(new PulsarProducerFactory(client), context.getProperty(MAX_PRODUCERS).asInteger()); + consumers = new ResourcePoolImpl(new PulsarConsumerFactory(client), context.getProperty(MAX_CONSUMERS).asInteger()); + + } + + private void createClient(final ConfigurationContext context) throws InitializationException { + + // We can't create a client without a service URL. + if (!context.getProperty(PULSAR_SERVICE_URL).isSet()) { + return; + } + + try { + this.client = PulsarClient.create(buildPulsarBrokerRootUrl(context.getProperty(PULSAR_SERVICE_URL).getValue(), + getClientConfig(context).isUseTls()), getClientConfig(context)); + + } catch (Exception e) { + throw new InitializationException("Unable to create Pulsar Client", e); + } + + } + + private static String buildPulsarBrokerRootUrl(String uri, boolean tlsEnabled) { + StringBuilder builder = new StringBuilder(); + builder.append("pulsar"); + + if (tlsEnabled) + builder.append("+ssl"); + + builder.append("://"); + builder.append(uri); + return builder.toString(); + } + + private ClientConfiguration getClientConfig(ConfigurationContext context) throws UnsupportedAuthenticationException { + + if (clientConfig == null) { + clientConfig = new ClientConfiguration(); + + if (context.getProperty(CONCURRENT_LOOKUP_REQUESTS).isSet()) { + clientConfig.setConcurrentLookupRequest(context.getProperty(CONCURRENT_LOOKUP_REQUESTS).asInteger()); + } + + if (context.getProperty(CONNECTIONS_PER_BROKER).isSet()) { + clientConfig.setConnectionsPerBroker(context.getProperty(CONNECTIONS_PER_BROKER).asInteger()); + } + + if (context.getProperty(IO_THREADS).isSet()) { + clientConfig.setIoThreads(context.getProperty(IO_THREADS).asInteger()); + } + + if (context.getProperty(LISTENER_THREADS).isSet()) { + clientConfig.setListenerThreads(context.getProperty(LISTENER_THREADS).asInteger()); + } + + if (context.getProperty(MAXIMUM_REJECTED_REQUESTS).isSet()) { + clientConfig.setMaxNumberOfRejectedRequestPerConnection(context.getProperty(MAXIMUM_REJECTED_REQUESTS).asInteger()); + } + + if (context.getProperty(OPERATION_TIMEOUT).isSet()) { + clientConfig.setOperationTimeout(context.getProperty(OPERATION_TIMEOUT).asInteger(), TimeUnit.SECONDS); + } + + if (context.getProperty(STATS_INTERVAL).isSet()) { + clientConfig.setStatsInterval(context.getProperty(STATS_INTERVAL).asLong(), TimeUnit.SECONDS); + } + + if (context.getProperty(USE_TCP_NO_DELAY).isSet()) { + clientConfig.setUseTcpNoDelay(context.getProperty(USE_TCP_NO_DELAY).asBoolean()); + } + + // Configure TLS + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + + if (sslContextService != null && sslContextService.isTrustStoreConfigured() && sslContextService.isKeyStoreConfigured()) { + clientConfig.setUseTls(true); + clientConfig.setTlsTrustCertsFilePath(sslContextService.getTrustStoreFile()); + + Map authParams = new HashMap<>(); + + // TODO This should be a different value than the TlsTrustCertsFilePath above. + authParams.put("tlsCertFile", sslContextService.getTrustStoreFile()); + authParams.put("tlsKeyFile", sslContextService.getKeyStoreFile()); + clientConfig.setAuthentication(AuthenticationTls.class.getName(), authParams); + } + } + + return clientConfig; + } + + @Override + public ResourcePool getProducerPool() { + return this.producers; + } + + @Override + public ResourcePool getConsumerPool() { + return this.consumers; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java new file mode 100644 index 000000000000..86d4934278d2 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java @@ -0,0 +1,31 @@ +/* + * 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.pulsar.pool; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ResourceExceptionHandlerImpl implements ResourceExceptionHandler { + + private static Logger logger = LoggerFactory.getLogger(ResourceExceptionHandlerImpl.class); + + @Override + public void handle(Exception exc) { + logger.error("Unable to create Resource", exc); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java new file mode 100644 index 000000000000..90ddb95aca13 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java @@ -0,0 +1,139 @@ +/* + * 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.pulsar.pool; + +import java.util.Iterator; +import java.util.Properties; +import java.util.Vector; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + + +public class ResourcePoolImpl implements ResourcePool { + + private final Lock lock = new ReentrantLock(); + private final Condition poolAvailable = lock.newCondition(); + private int max_resources; + private final Vector pool; + + private final ResourceExceptionHandler resourceExceptionHandler; + private final ResourceFactory factory; + + public ResourcePoolImpl(ResourceFactory factory, int max_resources) { + this(factory, new ResourceExceptionHandlerImpl(), max_resources); + } + + public ResourcePoolImpl(ResourceFactory factory, ResourceExceptionHandler handler, int max_resources) { + lock.lock(); + try { + this.factory = factory; + this.resourceExceptionHandler = handler; + this.max_resources = max_resources; + this.pool = new Vector (max_resources); + } finally { + lock.unlock(); + } + } + + private R createResource(Properties props) { + R resource = null; + try { + + resource = factory.create(props); + + if (resource == null) + throw new ResourceCreationException("Unable to create resource"); + + } catch (Exception e) { + resourceExceptionHandler.handle(e); + } + return resource; + } + + + /* + * Shutdown the pool and release the resources + */ + public void close() { + + Iterator itr = pool.iterator(); + while (itr.hasNext()) { + itr.next().close(); + } + + } + + public boolean isEmpty() { + return (pool.isEmpty()); + } + + public boolean isFull() { + return (pool != null && pool.size() == max_resources); + } + + @Override + public R acquire(Properties props) throws InterruptedException { + lock.lock(); + try { + while (max_resources <= 0) { + poolAvailable.await(); + } + + + --max_resources; + + if (pool != null) { + int size = pool.size(); + if (size > 0) + return pool.remove(size - 1); + } + return createResource(props); + } finally { + lock.unlock(); + } + } + + @Override + public void evict(R resource) { + lock.lock(); + try { + + // Attempt to close the connection + if (!resource.isClosed()) + resource.close(); + + pool.removeElement(resource); + --max_resources; + poolAvailable.signal(); + } finally { + lock.unlock(); + } + } + + @Override + public void release(R resource) { + lock.lock(); + try { + pool.addElement(resource); + ++max_resources; + poolAvailable.signal(); + } finally { + lock.unlock(); + } + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 000000000000..c1eecc70a562 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,15 @@ +# 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. +org.apache.nifi.pulsar.StandardPulsarClientPool \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java new file mode 100644 index 000000000000..72d17006e83f --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java @@ -0,0 +1,45 @@ +/* + * 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.pulsar; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +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; + +public class TestProcessor extends AbstractProcessor { + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + } + + @Override + protected List getSupportedPropertyDescriptors() { + List propDescs = new ArrayList<>(); + propDescs.add(new PropertyDescriptor.Builder() + .name("StandardPulsarClientService test processor") + .description("StandardPulsarClientService test processor") + .identifiesControllerService(StandardPulsarClientPool.class) + .required(true) + .build()); + return propDescs; + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java new file mode 100644 index 000000000000..4b9982df30db --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java @@ -0,0 +1,44 @@ +/* + * 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.pulsar; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class TestStandardPulsarClientService { + + @Before + public void init() { + + } + + @Test + public void testService() throws InitializationException { + final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class); + final PulsarClientPool service = new StandardPulsarClientPool(); + runner.addControllerService("test-good", service); + + runner.setProperty(service, StandardPulsarClientPool.PULSAR_SERVICE_URL, "localhost:6667"); + // runner.enableControllerService(service); + + runner.assertValid(service); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java new file mode 100644 index 000000000000..a19fbad556d3 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java @@ -0,0 +1,146 @@ +/* + * 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.pulsar.pool; + +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Rule; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.stubbing.Answer; + +import static org.mockito.Mockito.*; + +import java.util.Properties; + +public class MockPulsarClientService extends AbstractControllerService implements PulsarClientPool { + + @Mock + PulsarClient mockClient; + + @Mock + ResourcePool mockProducerPool; + + @Mock + ResourcePool mockConsumerPool; + + @Mock + Producer mockProducer; + + @Mock + Consumer mockConsumer; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @SuppressWarnings("unchecked") + public MockPulsarClientService() { + this.mockClient = mock(PulsarClient.class); + mockProducerPool = mock(ResourcePool.class); + mockConsumerPool = mock(ResourcePool.class); + mockProducer = mock(Producer.class); + mockConsumer = mock(Consumer.class); + + try { + when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarProducer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getProducer(props); + } + } + ); + + when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarConsumer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getConsumer(props); + } + } + ); + } catch (InterruptedException ex) { + + } + + try { + when(mockClient.createProducer(anyString())).thenReturn(getMockProducer()); + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(null); + } catch (PulsarClientException e) { + e.printStackTrace(); + } + } + + + public Producer getMockProducer() { + return mockProducer; + } + + public PulsarClient getMockClient() { + return mockClient; + } + + public PulsarProducer getProducer(Properties props) { + String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); + try { + return new PulsarProducer(mockClient.createProducer(topic), topic); + } catch (PulsarClientException e) { + return null; + } + } + + public PulsarConsumer getConsumer(Properties props) { + String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); + String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); + try { + return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + } catch (PulsarClientException e) { + return null; + } + } + + @Override + public ResourcePool getProducerPool() { + return mockProducerPool; + } + + + @Override + public ResourcePool getConsumerPool() { + return mockConsumerPool; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java new file mode 100644 index 000000000000..397b53880351 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java @@ -0,0 +1,146 @@ +/* + * 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.pulsar.pool; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.junit.Test; + +public class ResourcePoolTest { + + private MockPulsarClientService pulsarClient = new MockPulsarClientService(); + + /*Checks the number of the resources in the Resource Table. + Resource is created only if needed.*/ + @Test + public void testIsEmptyBeforeResourceAcquired() throws Exception { + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + assertTrue(resPool.isEmpty()); + } + + /* Checks if the number of resources in the Resource Table is equal to the + * maximum number of resources declared at the Resource Pool creation time.*/ + @Test + public void testIsFull() throws Exception { + + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + } + + /* + * Checks to see if resources are left in the pool after they + * have been acquired, and then released. + */ + @Test + public void testResourcesReused() throws InterruptedException { + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + for (int i = 0; i <= 4; i++) { + resPool.release(res[i]); + } + + assertFalse(resPool.isEmpty()); + } + + /* + * Checks to see if resources that are invalidated are removed from + * the pool. + */ + @Test + public void testResourcesReleased() throws InterruptedException { + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + for (int i = 0; i <= 4; i++) { + resPool.evict(res[i]); + } + + assertTrue(resPool.isEmpty()); + } + + @Test + public void testAcquireBlocksWhenEmpty() throws Exception { + + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "A"); + + final ResourcePoolImpl resPool = new ResourcePoolImpl( + new PulsarProducerFactory(pulsarClient.getMockClient()) , 0); + + Thread resourceConsumer = new Thread() { + public void run() { + try { + @SuppressWarnings("unused") + PulsarProducer unused = resPool.acquire(props); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + fail(); // error if control flow reaches this line + } + }; + + resourceConsumer.start(); + Thread.sleep(1000); // waits for the resourceConsumer to block + resourceConsumer.interrupt(); + resourceConsumer.join(1000); // resume after the resourceConsumer ends + assertFalse(resourceConsumer.isAlive()); + + } + + + private ResourcePoolImpl getResourcePool(String topic, int size) { + return new ResourcePoolImpl( + new PulsarProducerFactory(pulsarClient.getMockClient()), size); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml new file mode 100644 index 000000000000..680d08a13457 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml @@ -0,0 +1,37 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 1.5.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + pom + + + nifi-pulsar-client-service-api + nifi-pulsar-client-service-api-nar + nifi-pulsar-client-service + nifi-pulsar-client-service-nar + + + From b8b9601f2469df811b59b1019f0b7052dff9abd3 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Mon, 26 Feb 2018 11:19:26 -0800 Subject: [PATCH 02/16] Updated all nifi component version references to 1.6.0-SNAPSHOT --- .../nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml | 6 +++--- .../nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml | 4 ++-- nifi-nar-bundles/nifi-pulsar-bundle/pom.xml | 4 ++-- .../nifi-pulsar-client-service-api-nar/pom.xml | 6 +++--- .../nifi-pulsar-client-service-api/pom.xml | 2 +- .../nifi-pulsar-client-service-nar/pom.xml | 8 ++++---- .../nifi-pulsar-client-service/pom.xml | 4 ++-- nifi-nar-bundles/nifi-pulsar-client-services/pom.xml | 4 ++-- 8 files changed, 19 insertions(+), 19 deletions(-) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml index ca78a838211f..d986bf50fb98 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-bundle - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-nar @@ -33,13 +33,13 @@ org.apache.nifi nifi-pulsar-processors - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi nifi-pulsar-client-service-api-nar - 1.5.0 + 1.6.0-SNAPSHOT nar diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml index 5ebafd05e6fa..2931a50e918f 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-bundle - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-processors @@ -49,7 +49,7 @@ org.apache.nifi nifi-pulsar-client-service-api - 1.5.0 + 1.6.0-SNAPSHOT provided diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml index ccb90f23350b..abb745eb259f 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml @@ -19,12 +19,12 @@ org.apache.nifi nifi-nar-bundles - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi nifi-pulsar-bundle - 1.5.0 + 1.6.0-SNAPSHOT pom diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml index a0bd2122c2ae..4e6ed6d98a03 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml @@ -19,11 +19,11 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service-api-nar - 1.5.0 + 1.6.0-SNAPSHOT nar true @@ -39,7 +39,7 @@ org.apache.nifi nifi-pulsar-client-service-api - 1.5.0 + 1.6.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml index 2218c3258649..2259fc94f5ff 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service-api diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml index cb3c116e3772..460e2e12febf 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml @@ -19,11 +19,11 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service-nar - 1.5.0 + 1.6.0-SNAPSHOT nar true @@ -34,13 +34,13 @@ org.apache.nifi nifi-pulsar-client-service-api-nar - 1.5.0 + 1.6.0-SNAPSHOT nar org.apache.nifi nifi-pulsar-client-service - 1.5.0 + 1.6.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml index 366b9e66d4b8..5c3d3a6efc49 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service @@ -29,7 +29,7 @@ org.apache.nifi nifi-pulsar-client-service-api - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml index 680d08a13457..37b8e3fa3e54 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml @@ -19,12 +19,12 @@ org.apache.nifi nifi-nar-bundles - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT pom From d4e82ecc49e62596264db206f26b3b6a93f4b021 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Wed, 28 Feb 2018 10:18:35 -0800 Subject: [PATCH 03/16] Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 --- ...sumePulsar.java => ConsumePulsar_1_0.java} | 2 +- ...lishPulsar.java => PublishPulsar_1_0.java} | 2 +- .../org.apache.nifi.processor.Processor | 4 +- .../pulsar/AbstractPulsarProcessorTest.java | 2 +- .../pulsar/ConsumePulsarProcessorTest.java | 24 +++++----- .../pulsar/PublishPulsarProcessorTest.java | 44 +++++++++---------- .../apache/nifi/pulsar/PulsarClientPool.java | 14 ------ 7 files changed, 39 insertions(+), 53 deletions(-) rename nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/{ConsumePulsar.java => ConsumePulsar_1_0.java} (99%) rename nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/{PublishPulsar.java => PublishPulsar_1_0.java} (99%) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java similarity index 99% rename from nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java rename to nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java index b84cb7aa70c5..f331b11d4944 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java @@ -58,7 +58,7 @@ @CapabilityDescription("Consumes messages from Apache Pulsar " + "The complementary NiFi processor for sending messages is PublishPulsar.") @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) -public class ConsumePulsar extends AbstractPulsarProcessor { +public class ConsumePulsar_1_0 extends AbstractPulsarProcessor { static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java similarity index 99% rename from nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java rename to nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java index 20b93ddfd234..028d7061d00d 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java @@ -63,7 +63,7 @@ @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " + "FlowFiles that are routed to success.") -public class PublishPulsar extends AbstractPulsarProcessor { +public class PublishPulsar_1_0 extends AbstractPulsarProcessor { protected static final String MSG_COUNT = "msg.count"; diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index 93e412f3fc13..fbab5769250f 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -12,5 +12,5 @@ # 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. -org.apache.nifi.processors.pulsar.ConsumePulsar -org.apache.nifi.processors.pulsar.PublishPulsar \ No newline at end of file +org.apache.nifi.processors.pulsar.ConsumePulsar_1_0 +org.apache.nifi.processors.pulsar.PublishPulsar_1_0 \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java index c11008209f63..1bee9dc120a5 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -32,6 +32,6 @@ protected void addPulsarClientService() throws InitializationException { final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); runner.addControllerService("pulsarClient", pulsarClient); runner.enableControllerService(pulsarClient); - runner.setProperty(PublishPulsar.PULSAR_CLIENT_SERVICE, "pulsarClient"); + runner.setProperty(PublishPulsar_1_0.PULSAR_CLIENT_SERVICE, "pulsarClient"); } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java index 64550d442043..84ae6a4a56e2 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -49,7 +49,7 @@ public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { - runner = TestRunners.newTestRunner(ConsumePulsar.class); + runner = TestRunners.newTestRunner(ConsumePulsar_1_0.class); mockClient = mock(PulsarClient.class); mockConsumer = mock(Consumer.class); @@ -77,10 +77,10 @@ public void init() throws InitializationException { public void emptyMessageTest() { when(mockMessage.getData()).thenReturn("".getBytes()); - runner.setProperty(ConsumePulsar.TOPIC, "foo"); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); runner.run(); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); } @Test @@ -110,10 +110,10 @@ public void multipleAsyncMessagesTest() throws PulsarClientException { public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); - runner.setProperty(ConsumePulsar.TOPIC, "foo"); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); runner.run(10, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); runner.assertQueueEmpty(); @@ -132,14 +132,14 @@ private void sendMessages(String msg, String topic, String sub, boolean async, i when(mockMessage.getData()).thenReturn(msg.getBytes()); - runner.setProperty(ConsumePulsar.ASYNC_ENABLED, Boolean.toString(async)); - runner.setProperty(ConsumePulsar.TOPIC, topic); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, sub); + runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); runner.run(itertions, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); - List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar.REL_SUCCESS); + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_0.REL_SUCCESS); assertEquals(itertions, flowFiles.size()); for (MockFlowFile ff : flowFiles) { diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java index 30717a649e68..0b15b0ae5ec8 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -44,7 +44,7 @@ public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { - runner = TestRunners.newTestRunner(PublishPulsar.class); + runner = TestRunners.newTestRunner(PublishPulsar_1_0.class); mockClient = mock(PulsarClient.class); mockProducer = mock(Producer.class); @@ -82,15 +82,15 @@ public boolean matches(Object argument) { @Test public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); final String content = "some content"; Map attributes = new HashMap (); - attributes.put(PublishPulsar.TOPIC.getName(), ""); + attributes.put(PublishPulsar_1_0.TOPIC.getName(), ""); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_FAILURE); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); // Confirm that no Producer as created verify(mockClient, times(0)).createProducer(anyString()); @@ -99,15 +99,15 @@ public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); final String content = "some content"; Map attributes = new HashMap (); - attributes.put(PublishPulsar.TOPIC.getName(), "topic-b"); + attributes.put(PublishPulsar_1_0.TOPIC.getName(), "topic-b"); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); // Verify that we sent the data to topic-b. verify(mockClient, times(1)).createProducer("topic-b"); @@ -116,14 +116,14 @@ public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -136,15 +136,15 @@ public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClie @Test public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -157,14 +157,14 @@ public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, Pulsa @Test public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); } // Verify that the send method on the producer was called with the expected content @@ -174,15 +174,15 @@ public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarC @Test public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); } // Verify that the send method on the producer was called with the expected content @@ -191,14 +191,14 @@ public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, Pu @Test public void stressTest() throws UnsupportedEncodingException { - runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; for (int idx = 0; idx < 9999; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); } } diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java index be57a42a0d8c..be619eae4297 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java @@ -27,20 +27,6 @@ + "properties defined") public interface PulsarClientPool extends ControllerService { - /* - public PulsarProducer getProducer(Properties properties) throws PulsarClientException; - - public void release(PulsarProducer producer); - - public void invalidate(PulsarProducer producer); - - public PulsarConsumer getConsumer(Properties properties) throws PulsarClientException; - - public void release(PulsarConsumer consumer); - - public void invalidate(PulsarConsumer consumer); -*/ - public ResourcePool getProducerPool(); public ResourcePool getConsumerPool(); From 8b24b2496d404a65e6427273395ae73e558a5531 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 23 Feb 2018 17:40:22 -0800 Subject: [PATCH 04/16] Added Pulsar processors and Controller Service --- .../nifi-pulsar-bundle/.gitignore | 5 + .../nifi-pulsar-nar/pom.xml | 48 +++ .../nifi-pulsar-processors/pom.xml | 76 ++++ .../pulsar/AbstractPulsarProcessor.java | 43 ++ .../nifi/processors/pulsar/ConsumePulsar.java | 391 ++++++++++++++++++ .../nifi/processors/pulsar/PublishPulsar.java | 372 +++++++++++++++++ .../org.apache.nifi.processor.Processor | 16 + .../pulsar/AbstractPulsarProcessorTest.java | 37 ++ .../pulsar/ConsumePulsarProcessorTest.java | 162 ++++++++ .../pulsar/MockPulsarClientService.java | 156 +++++++ .../pulsar/PublishPulsarProcessorTest.java | 206 +++++++++ nifi-nar-bundles/nifi-pulsar-bundle/pom.xml | 35 ++ .../nifi-pulsar-client-services/.gitignore | 5 + .../pom.xml | 46 +++ .../nifi-pulsar-client-service-api/pom.xml | 40 ++ .../apache/nifi/pulsar/PulsarClientPool.java | 47 +++ .../apache/nifi/pulsar/PulsarConsumer.java | 70 ++++ .../apache/nifi/pulsar/PulsarProducer.java | 64 +++ .../apache/nifi/pulsar/cache/LRUCache.java | 69 ++++ .../nifi/pulsar/pool/PoolableResource.java | 25 ++ .../pulsar/pool/PulsarConsumerFactory.java | 56 +++ .../pulsar/pool/PulsarProducerFactory.java | 53 +++ .../pool/ResourceCreationException.java | 31 ++ .../pulsar/pool/ResourceExceptionHandler.java | 23 ++ .../nifi/pulsar/pool/ResourceFactory.java | 24 ++ .../apache/nifi/pulsar/pool/ResourcePool.java | 43 ++ .../nifi/pulsar/cache/LRUCacheTest.java | 135 ++++++ .../nifi-pulsar-client-service-nar/pom.xml | 47 +++ .../nifi-pulsar-client-service/pom.xml | 64 +++ .../nifi/pulsar/StandardPulsarClientPool.java | 300 ++++++++++++++ .../pool/ResourceExceptionHandlerImpl.java | 31 ++ .../nifi/pulsar/pool/ResourcePoolImpl.java | 139 +++++++ ...g.apache.nifi.controller.ControllerService | 15 + .../org/apache/nifi/pulsar/TestProcessor.java | 45 ++ .../TestStandardPulsarClientService.java | 44 ++ .../pulsar/pool/MockPulsarClientService.java | 146 +++++++ .../nifi/pulsar/pool/ResourcePoolTest.java | 146 +++++++ .../nifi-pulsar-client-services/pom.xml | 37 ++ 38 files changed, 3292 insertions(+) create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/.gitignore create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/.gitignore create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/pom.xml diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/.gitignore b/nifi-nar-bundles/nifi-pulsar-bundle/.gitignore new file mode 100644 index 000000000000..8cfc6f727089 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/.gitignore @@ -0,0 +1,5 @@ +.project +.settings/ +*/.classpath +*/.gitignore +/target/ diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml new file mode 100644 index 000000000000..ca78a838211f --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml @@ -0,0 +1,48 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-bundle + 1.5.0 + + + nifi-pulsar-nar + nar + + true + true + + + + + org.apache.nifi + nifi-pulsar-processors + 1.5.0 + + + + org.apache.nifi + nifi-pulsar-client-service-api-nar + 1.5.0 + nar + + + + + diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml new file mode 100644 index 000000000000..5ebafd05e6fa --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml @@ -0,0 +1,76 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-bundle + 1.5.0 + + + nifi-pulsar-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-record-serialization-service-api + + + org.apache.nifi + nifi-record + + + org.apache.nifi + nifi-utils + + + org.apache.nifi + nifi-ssl-context-service-api + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.5.0 + provided + + + org.apache.pulsar + pulsar-client + 1.21.0-incubating + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java new file mode 100644 index 000000000000..a2ecf430ba0f --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.pulsar.PulsarClientPool; + +public abstract class AbstractPulsarProcessor extends AbstractProcessor { + + protected static final PropertyDescriptor PULSAR_CLIENT_SERVICE = new PropertyDescriptor.Builder() + .name("Pulsar Client Service") + .description("Specified the Pulsar Client Service that can be used to create Pulsar connections") + .required(true) + .identifiesControllerService(PulsarClientPool.class) + .build(); + + protected static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles for which all content was sent to Pulsar.") + .build(); + + protected static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("Any FlowFile that cannot be sent to Pulsar will be routed to this Relationship") + .build(); +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java new file mode 100644 index 000000000000..b84cb7aa70c5 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerConfiguration; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionType; + +@Tags({"Pulsar", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume"}) +@CapabilityDescription("Consumes messages from Apache Pulsar " + + "The complementary NiFi processor for sending messages is PublishPulsar.") +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +public class ConsumePulsar extends AbstractPulsarProcessor { + + static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); + static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); + static final AllowableValue FAILOVER = new AllowableValue("Failover", "Failover", "Multiple consumer will be able to use the same subscription name but only 1 consumer " + + "will receive the messages. If that consumer disconnects, one of the other connected consumers will start receiving messages"); + + protected static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + static final PropertyDescriptor SUBSCRIPTION = new PropertyDescriptor.Builder() + .name("Subscription") + .displayName("Subscription Name") + .description("The name of the Pulsar subscription to consume from.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + .name("Async Enabled") + .description("Control whether the messages will be consumed asyncronously or not. Messages consumed" + + " syncronously will be acknowledged immediately before processing the next message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum Async Requests") + .description("The maximum number of outstanding asynchronous consumer requests for this processor. " + + "Each asynchronous call requires memory, so avoid setting this value to high.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("50") + .build(); + + public static final PropertyDescriptor ACK_TIMEOUT = new PropertyDescriptor.Builder() + .name("Acknowledgment Timeout") + .description("Set the timeout (in milliseconds) for unacked messages, truncated to the " + + "nearest millisecond. The timeout needs to be greater than 10 seconds.") + .required(false) + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .defaultValue("10000") + .build(); + + public static final PropertyDescriptor PRIORITY_LEVEL = new PropertyDescriptor.Builder() + .name("Consumer Priority Level") + .description("Sets priority level for the shared subscription consumers to which broker " + + "gives more priority while dispatching messages. Here, broker follows descending " + + "priorities. (eg: 0=max-priority, 1, 2,..) ") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("5") + .build(); + + public static final PropertyDescriptor RECEIVER_QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Consumer receive queue size.") + .description("The consumer receive queue controls how many messages can be accumulated " + + "by the Consumer before the application calls Consumer.receive(). Using a higher " + + "value could potentially increase the consumer throughput at the expense of bigger " + + "memory utilization. \n" + + "Setting the consumer queue size as zero, \n" + + "\t - Decreases the throughput of the consumer, by disabling pre-fetching of messages. \n" + + "\t - Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer " + + "connection with broker and consumer will not be able receive any further message unless batch-message " + + "in pipeline is removed") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + public static final PropertyDescriptor SUBSCRIPTION_TYPE = new PropertyDescriptor.Builder() + .name("Subscription Type") + .description("Select the subscription type to be used when subscribing to the topic.") + .required(false) + .allowableValues(EXCLUSIVE, SHARED, FAILOVER) + .defaultValue(SHARED.getValue()) + .build(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + // Reuse the same consumer for a given topic / subscription + private PulsarConsumer consumer; + private ConsumerConfiguration consumerConfig; + + // Pool for running multiple consume Async requests + ExecutorService pool; + ExecutorCompletionService completionService; + + static { + final List properties = new ArrayList<>(); + properties.add(PULSAR_CLIENT_SERVICE); + properties.add(TOPIC); + properties.add(SUBSCRIPTION); + properties.add(ASYNC_ENABLED); + properties.add(MAX_ASYNC_REQUESTS); + properties.add(ACK_TIMEOUT); + properties.add(PRIORITY_LEVEL); + properties.add(RECEIVER_QUEUE_SIZE); + properties.add(SUBSCRIPTION_TYPE); + + PROPERTIES = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + RELATIONSHIPS = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTIES; + } + + @OnScheduled + public void init(ProcessContext context) { + pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); + completionService = new ExecutorCompletionService<>(pool); + } + + @OnUnscheduled + public void shutDown() { + // Stop all the async consumers + pool.shutdownNow(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + try { + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + // Launch consumers + consumeAsync(context, session); + + // Handle completed consumers + handleAsync(context, session); + + } else { + consume(context, session); + } + } catch (PulsarClientException e) { + getLogger().error("Unable to consume from Pulsar Topic ", e); + context.yield(); + throw new ProcessException(e); + } + + } + + private void handleAsync(ProcessContext context, ProcessSession session) { + + try { + Future done = completionService.take(); + Message msg = done.get(); + + if (msg != null) { + FlowFile flowFile = null; + final byte[] value = msg.getData(); + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + } + + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.transfer(flowFile, REL_SUCCESS); + session.commit(); + getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); + } + + } catch (InterruptedException | ExecutionException | PulsarClientException e) { + getLogger().error("Trouble consuming messages ", e); + } + + } + + @OnStopped + public void close(final ProcessContext context) { + + getLogger().info("Disconnecting Pulsar Consumer"); + if (consumer != null) { + + context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getConsumerPool().evict(consumer); + } + + consumer = null; + } + + /* + * For now let's assume that this processor will be configured to run for a longer + * duration than 0 milliseconds. So we will be grabbing as many messages off the topic + * as possible and committing them as FlowFiles + */ + private void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException { + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + completionService.submit(new Callable() { + @Override + public Message call() throws Exception { + return consumer.receiveAsync().get(); + } + }); + + } + + /* + * When this Processor expects to receive many small files, it may + * be advisable to create several FlowFiles from a single session + * before committing the session. Typically, this allows the Framework + * to treat the content of the newly created FlowFiles much more efficiently. + */ + private void consume(ProcessContext context, ProcessSession session) throws PulsarClientException { + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + final ComponentLog logger = getLogger(); + final Message msg; + FlowFile flowFile = null; + + try { + + msg = consumer.receive(); + final byte[] value = msg.getData(); + + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.transfer(flowFile, REL_SUCCESS); + logger.info("Created {} from {} messages received from Pulsar Server and transferred to 'success'", + new Object[]{flowFile, 1}); + + session.commit(); + + /* + * This Processor acknowledges receipt of the data and/or removes the data + * from the external source in order to prevent receipt of duplicate files. + * This is done only after the ProcessSession by which the FlowFile was created + * has been committed! Failure to adhere to this principle may result in data + * loss, as restarting NiFi before the session has been committed will result + * in the temporary file being deleted. Note, however, that it is possible using + * this approach to receive duplicate data because the application could be + * restarted after committing the session and before acknowledging or removing + * the data from the external source. In general, though, potential data duplication + * is preferred over potential data loss. + */ + getLogger().info("Acknowledging message " + msg.getMessageId()); + consumer.acknowledge(msg); + + } else { + // We didn't consume any data, so + session.commit(); + } + + } catch (PulsarClientException e) { + context.yield(); + session.rollback(); + } + + } + + private PulsarConsumer getWrappedConsumer(ProcessContext context) throws PulsarClientException { + + if (consumer != null) + return consumer; + + final PulsarClientPool pulsarClientService = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class); + + try { + consumer = pulsarClientService.getConsumerPool() + .acquire(getConsumerProperties(context)); + + if (consumer == null || consumer.getConsumer() == null) { + throw new PulsarClientException("Unable to create Pulsar Consumer"); + } + + return consumer; + } catch (final InterruptedException ex) { + return null; + } + } + + private Properties getConsumerProperties(ProcessContext context) { + + Properties props = new Properties(); + props.put(PulsarConsumerFactory.TOPIC_NAME, context.getProperty(TOPIC).getValue()); + props.put(PulsarConsumerFactory.SUBSCRIPTION_NAME, context.getProperty(SUBSCRIPTION).getValue()); + props.put(PulsarConsumerFactory.CONSUMER_CONFIG, getConsumerConfig(context)); + return props; + } + + private ConsumerConfiguration getConsumerConfig(ProcessContext context) { + + if (consumerConfig == null) { + consumerConfig = new ConsumerConfiguration(); + + if (context.getProperty(ACK_TIMEOUT).isSet()) + consumerConfig.setAckTimeout(context.getProperty(ACK_TIMEOUT).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(PRIORITY_LEVEL).isSet()) + consumerConfig.setPriorityLevel(context.getProperty(PRIORITY_LEVEL).asInteger()); + + if (context.getProperty(RECEIVER_QUEUE_SIZE).isSet()) + consumerConfig.setReceiverQueueSize(context.getProperty(RECEIVER_QUEUE_SIZE).asInteger()); + + if (context.getProperty(SUBSCRIPTION_TYPE).isSet()) + consumerConfig.setSubscriptionType(SubscriptionType.valueOf(context.getProperty(SUBSCRIPTION_TYPE).getValue())); + } + + return consumerConfig; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java new file mode 100644 index 000000000000..20b93ddfd234 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.cache.LRUCache; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.util.StringUtils; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConfiguration; +import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode; +import org.apache.pulsar.client.api.PulsarClientException; + +@Tags({"Apache", "Pulsar", "Put", "Send", "Message", "PubSub"}) +@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Pulsar using the Pulsar 1.21 Producer API." + + "The messages to send may be individual FlowFiles or may be delimited, using a " + + "user-specified delimiter, such as a new-line. " + + "The complementary NiFi processor for fetching messages is ConsumePulsar.") +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " + + "FlowFiles that are routed to success.") +public class PublishPulsar extends AbstractPulsarProcessor { + + protected static final String MSG_COUNT = "msg.count"; + + static final AllowableValue COMPRESSION_TYPE_NONE = new AllowableValue("NONE", "None", "No compression"); + static final AllowableValue COMPRESSION_TYPE_LZ4 = new AllowableValue("LZ4", "LZ4", "Compress with LZ4 algorithm."); + static final AllowableValue COMPRESSION_TYPE_ZLIB = new AllowableValue("ZLIB", "ZLIB", "Compress with ZLib algorithm"); + + static final AllowableValue MESSAGE_ROUTING_MODE_CUSTOM_PARTITION = new AllowableValue("CustomPartition", "Custom Partition", "Route messages to a custom partition"); + static final AllowableValue MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION = new AllowableValue("RoundRobinPartition", "Round Robin Partition", "Route messages to all partitions in a round robin manner"); + static final AllowableValue MESSAGE_ROUTING_MODE_SINGLE_PARTITION = new AllowableValue("SinglePartition", "Single Partition", "Route messages to a single partition"); + + public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + .name("Async Enabled") + .description("Control whether the messages will be sent asyncronously or not. Messages sent" + + " syncronously will be acknowledged immediately before processing the next message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor BATCHING_ENABLED = new PropertyDescriptor.Builder() + .name("Batching Enabled") + .description("Control whether automatic batching of messages is enabled for the producer. " + + "default: false [No batching] When batching is enabled, multiple calls to " + + "Producer.sendAsync can result in a single batch to be sent to the broker, leading " + + "to better throughput, especially when publishing small messages. If compression is " + + "enabled, messages will be compressed at the batch level, leading to a much better " + + "compression ratio for similar headers or contents. When enabled default batch delay " + + "is set to 10 ms and default batch size is 1000 messages") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor BATCHING_MAX_MESSAGES = new PropertyDescriptor.Builder() + .name("Batching Max Messages") + .description("Set the maximum number of messages permitted in a batch. default: " + + "1000 If set to a value greater than 1, messages will be queued until this " + + "threshold is reached or batch interval has elapsed") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + public static final PropertyDescriptor BATCH_INTERVAL = new PropertyDescriptor.Builder() + .name("Batch Interval") + .description("Set the time period within which the messages sent will be batched default: 10ms " + + "if batch messages are enabled. If set to a non zero value, messages will be queued until " + + "this time interval or until the Batching Max Messages threshould has been reached") + .required(false) + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .defaultValue("10") + .build(); + + public static final PropertyDescriptor BLOCK_IF_QUEUE_FULL = new PropertyDescriptor.Builder() + .name("Block if Message Queue Full") + .description("Set whether the processor should block when the outgoing message queue is full. " + + "Default is false. If set to false, send operations will immediately fail with " + + "ProducerQueueIsFullError when there is no space left in pending queue.") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() + .name("Compression Type") + .description("Set the compression type for the producer.") + .required(false) + .allowableValues(COMPRESSION_TYPE_NONE, COMPRESSION_TYPE_LZ4, COMPRESSION_TYPE_ZLIB) + .defaultValue(COMPRESSION_TYPE_NONE.getValue()) + .build(); + + public static final PropertyDescriptor MESSAGE_ROUTING_MODE = new PropertyDescriptor.Builder() + .name("Message Routing Mode") + .description("Set the message routing mode for the producer. This applies only if the destination topic is partitioned") + .required(false) + .allowableValues(MESSAGE_ROUTING_MODE_CUSTOM_PARTITION, MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION, MESSAGE_ROUTING_MODE_SINGLE_PARTITION) + .defaultValue(MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION.getValue()) + .build(); + + public static final PropertyDescriptor PENDING_MAX_MESSAGES = new PropertyDescriptor.Builder() + .name("Max Pending Messages") + .description("Set the max size of the queue holding the messages pending to receive an " + + "acknowledgment from the broker.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + private LRUCache producers; + private ProducerConfiguration producerConfig; + + static { + final List properties = new ArrayList<>(); + properties.add(PULSAR_CLIENT_SERVICE); + properties.add(TOPIC); + properties.add(ASYNC_ENABLED); + properties.add(BATCHING_ENABLED); + properties.add(BATCHING_MAX_MESSAGES); + properties.add(BATCH_INTERVAL); + properties.add(BLOCK_IF_QUEUE_FULL); + properties.add(COMPRESSION_TYPE); + properties.add(MESSAGE_ROUTING_MODE); + properties.add(PENDING_MAX_MESSAGES); + + PROPERTIES = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + RELATIONSHIPS = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTIES; + } + + @OnStopped + public void cleanUp(final ProcessContext context) { + // Close all of the producers and invalidate them, so they get removed from the Resource Pool + getProducerCache(context).clear(); + } + + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + FlowFile flowFile = session.get(); + + if (flowFile == null) + return; + + final ComponentLog logger = getLogger(); + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + + if (StringUtils.isBlank(topic)) { + logger.error("Invalid topic specified {}", new Object[] {topic}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + // Read the contents of the FlowFile into a byte array + final byte[] messageContent = new byte[(int) flowFile.getSize()]; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + StreamUtils.fillBuffer(in, messageContent, true); + } + }); + + // Nothing to do, so skip this Flow file. + if (messageContent == null || messageContent.length < 1) { + session.transfer(flowFile, REL_SUCCESS); + return; + } + + try { + + Producer producer = getWrappedProducer(topic, context).getProducer(); + + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + this.sendAsync(producer, session, flowFile, messageContent); + } else { + this.send(producer, session, flowFile, messageContent); + } + + } catch (final PulsarClientException e) { + logger.error("Failed to connect to Pulsar Server due to {}", new Object[]{e}); + session.penalize(flowFile); + session.transfer(flowFile, REL_FAILURE); + } + + } + + + private void send(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) throws PulsarClientException { + + MessageId msgId = producer.send(messageContent); + + if (msgId != null) { + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } else { + session.transfer(flowFile, REL_FAILURE); + } + + } + + private void sendAsync(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) { + + producer.sendAsync(messageContent).handle((msgId, ex) -> { + if (msgId != null) { + return msgId; + } else { + // TODO Communicate the error back up to the onTrigger method so we can invalidate this producer. + getLogger().warn("Problem ", ex); + return null; + } + }); + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } + + private PulsarProducer getWrappedProducer(String topic, ProcessContext context) throws PulsarClientException, IllegalArgumentException { + + PulsarProducer producer = getProducerCache(context).get(topic); + + if (producer != null) + return producer; + + try { + producer = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool().acquire(getProducerProperties(context, topic)); + + if (producer != null) { + producers.put(topic, producer); + } + + return producer; + + } catch (InterruptedException e) { + return null; + } + + } + + private LRUCache getProducerCache(ProcessContext context) { + if (producers == null) { + + ResourcePool pool = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool(); + + producers = new LRUCache (20, pool); + } + + return producers; + } + + private Properties getProducerProperties(ProcessContext context, String topic) { + + Properties props = new Properties(); + props.put(PulsarProducerFactory.TOPIC_NAME, topic); + props.put(PulsarProducerFactory.PRODUCER_CONFIG, getProducerConfig(context)); + return props; + } + + private ProducerConfiguration getProducerConfig(ProcessContext context) { + + if (producerConfig == null) { + producerConfig = new ProducerConfiguration(); + + if (context.getProperty(BATCHING_ENABLED).isSet()) + producerConfig.setBatchingEnabled(context.getProperty(BATCHING_ENABLED).asBoolean()); + + if (context.getProperty(BATCHING_MAX_MESSAGES).isSet()) + producerConfig.setBatchingMaxMessages(context.getProperty(BATCHING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(BATCH_INTERVAL).isSet()) + producerConfig.setBatchingMaxPublishDelay(context.getProperty(BATCH_INTERVAL).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(BLOCK_IF_QUEUE_FULL).isSet()) + producerConfig.setBlockIfQueueFull(context.getProperty(BLOCK_IF_QUEUE_FULL).asBoolean()); + + if (context.getProperty(COMPRESSION_TYPE).isSet()) + producerConfig.setCompressionType(CompressionType.valueOf(context.getProperty(COMPRESSION_TYPE).getValue())); + + if (context.getProperty(PENDING_MAX_MESSAGES).isSet()) + producerConfig.setMaxPendingMessages(context.getProperty(PENDING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(MESSAGE_ROUTING_MODE).isSet()) + producerConfig.setMessageRoutingMode(MessageRoutingMode.valueOf(context.getProperty(MESSAGE_ROUTING_MODE).getValue())); + } + + return producerConfig; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 000000000000..93e412f3fc13 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# 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. +org.apache.nifi.processors.pulsar.ConsumePulsar +org.apache.nifi.processors.pulsar.PublishPulsar \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java new file mode 100644 index 000000000000..c11008209f63 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.TestRunner; +import org.apache.pulsar.client.api.PulsarClient; +import org.mockito.Mock; + +public abstract class AbstractPulsarProcessorTest { + + protected TestRunner runner; + + @Mock + protected PulsarClient mockClient; + + protected void addPulsarClientService() throws InitializationException { + final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); + runner.addControllerService("pulsarClient", pulsarClient); + runner.enableControllerService(pulsarClient); + runner.setProperty(PublishPulsar.PULSAR_CLIENT_SERVICE, "pulsarClient"); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java new file mode 100644 index 000000000000..64550d442043 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunners; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.*; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { + + @Mock + Consumer mockConsumer; + + @Mock + Message mockMessage; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @Before + public void init() throws InitializationException { + runner = TestRunners.newTestRunner(ConsumePulsar.class); + + mockClient = mock(PulsarClient.class); + mockConsumer = mock(Consumer.class); + mockMessage = mock(Message.class); + + try { + when(mockClient.subscribe(anyString(), anyString())).thenReturn(mockConsumer); + when(mockConsumer.receive()).thenReturn(mockMessage); + + CompletableFuture future = CompletableFuture.supplyAsync(() -> { + return mockMessage; + }); + + when(mockConsumer.receiveAsync()).thenReturn(future); + + } catch (PulsarClientException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + + addPulsarClientService(); + } + + @Test + public void emptyMessageTest() { + when(mockMessage.getData()).thenReturn("".getBytes()); + + runner.setProperty(ConsumePulsar.TOPIC, "foo"); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.run(); + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + } + + @Test + public void singleSyncMessageTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", false, 1); + } + + @Test + public void multipleSyncMessagesTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", false, 40); + } + + @Test + public void singleAsyncMessageTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", true, 1); + } + + @Test + public void multipleAsyncMessagesTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", true, 40); + } + + /* + * Verify that the consumer gets closed. + */ + @Test + public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { + when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); + + runner.setProperty(ConsumePulsar.TOPIC, "foo"); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.run(10, true); + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + + runner.assertQueueEmpty(); + + // Verify that the receive method on the consumer was called 10 times + verify(mockConsumer, times(10)).receive(); + + // Verify that each message was acknowledged + verify(mockConsumer, times(10)).acknowledge(mockMessage); + + // Verify that the consumer was closed + verify(mockConsumer, times(1)).close(); + + } + + private void sendMessages(String msg, String topic, String sub, boolean async, int itertions) throws PulsarClientException { + + when(mockMessage.getData()).thenReturn(msg.getBytes()); + + runner.setProperty(ConsumePulsar.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar.TOPIC, topic); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, sub); + runner.run(itertions, true); + + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar.REL_SUCCESS); + assertEquals(itertions, flowFiles.size()); + + for (MockFlowFile ff : flowFiles) { + ff.assertContentEquals(msg); + } + + if (async) { + verify(mockConsumer, times(itertions)).receiveAsync(); + } else { + verify(mockConsumer, times(itertions)).receive(); + } + + // Verify that every message was acknowledged + if (async) { + verify(mockConsumer, times(itertions)).acknowledgeAsync(mockMessage); + } else { + verify(mockConsumer, times(itertions)).acknowledge(mockMessage); + } + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java new file mode 100644 index 000000000000..3f7829272eb0 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Rule; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.stubbing.Answer; + +import static org.mockito.Mockito.*; + +import java.util.Properties; + +public class MockPulsarClientService extends AbstractControllerService implements PulsarClientPool { + + @Mock + PulsarClient mockClient; + + @Mock + ResourcePool mockProducerPool; + + @Mock + ResourcePool mockConsumerPool; + + @Mock + Producer mockProducer; + + @Mock + Consumer mockConsumer; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @SuppressWarnings({ "unchecked", "rawtypes" }) + public MockPulsarClientService(PulsarClient mockClient2) { + this.mockClient = mockClient2; + mockProducerPool = mock(ResourcePool.class); + mockConsumerPool = mock(ResourcePool.class); + mockProducer = mock(Producer.class); + mockConsumer = mock(Consumer.class); + + try { + when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarProducer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getProducer(props); + } + } + ); + + when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarConsumer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getConsumer(props); + } + } + ); + + + doAnswer(new Answer() { + public Object answer(InvocationOnMock invocation){ + PulsarConsumer consumer = invocation.getArgumentAt(0, PulsarConsumer.class); + consumer.close(); + return null; + } + }).when(mockConsumerPool).evict(any(PulsarConsumer.class)); + + } catch (InterruptedException ex) { + + } + + try { + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(null); + } catch (PulsarClientException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + + + public Producer getMockProducer() { + return mockProducer; + } + + public PulsarClient getMockClient() { + return mockClient; + } + + public PulsarProducer getProducer(Properties props) { + String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); + try { + return new PulsarProducer(mockClient.createProducer(topic), topic); + } catch (PulsarClientException e) { + return null; + } + } + + public PulsarConsumer getConsumer(Properties props) { + String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); + String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); + try { + return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + } catch (PulsarClientException e) { + return null; + } + } + + @Override + public ResourcePool getProducerPool() { + return mockProducerPool; + } + + + @Override + public ResourcePool getConsumerPool() { + return mockConsumerPool; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java new file mode 100644 index 000000000000..30717a649e68 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.io.UnsupportedEncodingException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunners; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; +import org.mockito.Mock; + +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.*; + +public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { + + @Mock + Producer mockProducer; + + @Before + public void init() throws InitializationException { + runner = TestRunners.newTestRunner(PublishPulsar.class); + + mockClient = mock(PulsarClient.class); + mockProducer = mock(Producer.class); + + try { + // Use the mockProducer for all Producer interactions + when(mockClient.createProducer(anyString())).thenReturn(mockProducer); + + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(mock(MessageId.class)); + + CompletableFuture future = CompletableFuture.supplyAsync(() -> { + return mock(MessageId.class); + }); + + when(mockProducer.sendAsync(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(future); + + + } catch (PulsarClientException e) { + e.printStackTrace(); + } + + addPulsarClientService(); + } + + @Test + public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap (); + attributes.put(PublishPulsar.TOPIC.getName(), ""); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_FAILURE); + + // Confirm that no Producer as created + verify(mockClient, times(0)).createProducer(anyString()); + } + + @Test + public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap (); + attributes.put(PublishPulsar.TOPIC.getName(), "topic-b"); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + // Verify that we sent the data to topic-b. + verify(mockClient, times(1)).createProducer("topic-b"); + } + + @Test + public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + + final String content = "some content"; + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + outFile.assertContentEquals(content); + + // Verify that we sent the data to my-topic. + verify(mockClient, times(1)).createProducer("my-topic"); + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(1)).send(content.getBytes()); + } + + @Test + public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + + final String content = "some content"; + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + outFile.assertContentEquals(content); + + // Verify that we sent the data to my-topic. + verify(mockClient, times(1)).createProducer("my-topic"); + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(1)).sendAsync(content.getBytes()); + } + + @Test + public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + final String content = "some content"; + + // Hack, since runner.run(20, false); doesn't work as advertised + for (int idx = 0; idx < 20; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + } + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(20)).send(content.getBytes()); + } + + @Test + public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + final String content = "some content"; + + // Hack, since runner.run(20, false); doesn't work as advertised + for (int idx = 0; idx < 20; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + } + + // Verify that the send method on the producer was called with the expected content + verify(mockProducer, times(20)).sendAsync(content.getBytes()); + } + + @Test + public void stressTest() throws UnsupportedEncodingException { + runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + final String content = "some content"; + + for (int idx = 0; idx < 9999; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml new file mode 100644 index 000000000000..ccb90f23350b --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml @@ -0,0 +1,35 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 1.5.0 + + + org.apache.nifi + nifi-pulsar-bundle + 1.5.0 + pom + + + nifi-pulsar-processors + nifi-pulsar-nar + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore b/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore new file mode 100644 index 000000000000..8cfc6f727089 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore @@ -0,0 +1,5 @@ +.project +.settings/ +*/.classpath +*/.gitignore +/target/ diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml new file mode 100644 index 000000000000..a0bd2122c2ae --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml @@ -0,0 +1,46 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service-api-nar + 1.5.0 + nar + + true + true + + + + + org.apache.nifi + nifi-standard-services-api-nar + nar + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.5.0 + + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml new file mode 100644 index 000000000000..2218c3258649 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service-api + jar + + + + org.apache.nifi + nifi-api + provided + + + org.apache.pulsar + pulsar-client + 1.21.0-incubating + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java new file mode 100644 index 000000000000..be57a42a0d8c --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java @@ -0,0 +1,47 @@ +/* + * 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.pulsar; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.pulsar.pool.ResourcePool; + + +@Tags({"Pulsar"}) +@CapabilityDescription("Provides the ability to create Pulsar Producer / Consumer instances on demand, based on the configuration." + + "properties defined") +public interface PulsarClientPool extends ControllerService { + + /* + public PulsarProducer getProducer(Properties properties) throws PulsarClientException; + + public void release(PulsarProducer producer); + + public void invalidate(PulsarProducer producer); + + public PulsarConsumer getConsumer(Properties properties) throws PulsarClientException; + + public void release(PulsarConsumer consumer); + + public void invalidate(PulsarConsumer consumer); +*/ + + public ResourcePool getProducerPool(); + + public ResourcePool getConsumerPool(); +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java new file mode 100644 index 000000000000..59c98cdd0246 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java @@ -0,0 +1,70 @@ +/* + * 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.pulsar; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.ConsumerStats; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("unused") +public class PulsarConsumer implements PoolableResource { + + private static final Logger logger = LoggerFactory.getLogger(PulsarConsumer.class); + + private final Consumer consumer; + private final String topicName; + private final String subscriptionName; + private boolean closed = false; + + public PulsarConsumer(Consumer consumer, String topic, String subscription) throws PulsarClientException { + this.consumer = consumer; + this.topicName = topic; + this.subscriptionName = subscription; + } + + public void close() { + + logger.info("Closing consumer for topic {} and subscription {}", new Object[] {topicName, subscriptionName}); + closed = true; + + try { + consumer.unsubscribe(); + consumer.close(); + } catch (PulsarClientException e) { + logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); + closed = false; + } + } + + public boolean isClosed() { + return closed; + } + + public Consumer getConsumer() { + return this.consumer; + } + + public ConsumerStats getStats() { + return this.consumer.getStats(); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java new file mode 100644 index 000000000000..a13975b270c2 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java @@ -0,0 +1,64 @@ +/* + * 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.pulsar; + +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PulsarProducer implements PoolableResource { + + private static final Logger logger = LoggerFactory.getLogger(PulsarProducer.class); + + private final Producer producer; + private boolean closed = false; + private final String topicName; + + public PulsarProducer(Producer producer, String topicName) throws PulsarClientException { + this.topicName = topicName; + this.producer = producer; + } + + public Producer getProducer() { + return producer; + } + + public String getName() { + return topicName; + } + + public boolean isClosed() { + return this.closed; + } + + public void close() { + + logger.info("Closing producer for topic {} ", new Object[] {topicName}); + + this.closed = true; + try { + producer.close(); + } catch (PulsarClientException e) { + logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); + closed = false; + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java new file mode 100644 index 000000000000..86a0fc2afab0 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java @@ -0,0 +1,69 @@ +/* + * 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.pulsar.cache; + +import java.util.LinkedHashMap; + +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.nifi.pulsar.pool.ResourcePool; + +public class LRUCache { + + private LinkedHashMap lruCacheMap; + private final int capacity; + private final boolean SORT_BY_ACCESS = true; + private final float LOAD_FACTOR = 0.75F; + private final ResourcePool resourcePool; + + public LRUCache(int capacity, ResourcePool resourcePool){ + this.capacity = capacity; + this.lruCacheMap = new LinkedHashMap<>(capacity, LOAD_FACTOR, SORT_BY_ACCESS); + this.resourcePool = resourcePool; + } + + public V get(K k){ + return lruCacheMap.get(k); + } + + public void put(K k, V v){ + if(lruCacheMap.containsKey(k)){ + lruCacheMap.remove(k); + } else if(lruCacheMap.size() >= capacity){ + K victimKey = lruCacheMap.keySet().iterator().next(); + V victim = lruCacheMap.get(victimKey); + lruCacheMap.remove(victimKey); + resourcePool.evict(victim); + } + lruCacheMap.put(k, v); + } + + public int getSize() { + return lruCacheMap.size(); + } + + public void clear() { + for (V victim :lruCacheMap.values()) { + resourcePool.evict(victim); + } + + lruCacheMap.clear(); + } + + public void printSequence(){ + System.out.println(lruCacheMap.keySet()); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java new file mode 100644 index 000000000000..6af5845e0885 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java @@ -0,0 +1,25 @@ +/* + * 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.pulsar.pool; + +public interface PoolableResource { + + public void close(); + + public boolean isClosed(); + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java new file mode 100644 index 000000000000..6794d2a7165e --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java @@ -0,0 +1,56 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.pulsar.client.api.ConsumerConfiguration; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; + +public class PulsarConsumerFactory implements ResourceFactory { + + public static final String TOPIC_NAME = "topic"; + public static final String SUBSCRIPTION_NAME = "subscription"; + public static final String CONSUMER_CONFIG = "Consumer-Configuration"; + + private PulsarClient client; + + public PulsarConsumerFactory(PulsarClient client) { + this.client = client; + } + + @Override + public PulsarConsumer create(Properties props) throws ResourceCreationException { + + String topic = props.getProperty(TOPIC_NAME); + String subscription = props.getProperty(SUBSCRIPTION_NAME); + ConsumerConfiguration config = (ConsumerConfiguration) props.get(CONSUMER_CONFIG); + + try { + // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice + return (config == null) ? new PulsarConsumer(client.subscribe(topic, subscription), topic, subscription) : + new PulsarConsumer(client.subscribe(topic, subscription, config), topic, subscription); + + } catch (PulsarClientException e) { + throw new ResourceCreationException(e); + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java new file mode 100644 index 000000000000..37165fc45ee6 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java @@ -0,0 +1,53 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.pulsar.client.api.ProducerConfiguration; +import org.apache.pulsar.client.api.PulsarClient; + +public class PulsarProducerFactory implements ResourceFactory { + + public static final String TOPIC_NAME = "topic"; + public static final String PRODUCER_CONFIG = "Producer-Configuration"; + + private PulsarClient client; + + public PulsarProducerFactory(PulsarClient client) { + this.client = client; + } + + @Override + public PulsarProducer create(Properties props) throws ResourceCreationException { + + String topic = props.getProperty(TOPIC_NAME); + ProducerConfiguration config = (ProducerConfiguration) props.get(PRODUCER_CONFIG); + + try { + // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice + return (config == null) ? new PulsarProducer(client.createProducer(topic), topic) : + new PulsarProducer(client.createProducer(topic, config), topic); + + } catch (Exception e) { + throw new ResourceCreationException(e); + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java new file mode 100644 index 000000000000..7eb3ea6ccee5 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java @@ -0,0 +1,31 @@ +/* + * 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.pulsar.pool; + +public class ResourceCreationException extends Exception { + + private static final long serialVersionUID = -1796731800483756134L; + + public ResourceCreationException(String s) { + super(s); + } + + public ResourceCreationException(Exception e) { + super(e); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java new file mode 100644 index 000000000000..403d95cbcd71 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java @@ -0,0 +1,23 @@ + +/* + * 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.pulsar.pool; + +public interface ResourceExceptionHandler { + + void handle(Exception exc); +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java new file mode 100644 index 000000000000..1cf0500d441d --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.pulsar.pool; + +import java.util.Properties; + +public interface ResourceFactory { + + public R create(Properties props) throws ResourceCreationException; +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java new file mode 100644 index 000000000000..e67b8f4b58e7 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java @@ -0,0 +1,43 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +public interface ResourcePool { + + /** + * Acquire a resource from the pool. Creating one if necessary + * @param props + * @return + * @throws InterruptedException + */ + public R acquire(Properties props) throws InterruptedException; + + /** + * Evict the resource from the pool, destroying it. + * Call this method is the resource is known to be in an unusable state. + * @param resource + */ + public void evict(R resource); + + /** + * Place the resource back into the pool for future use. + * @param resource + */ + public void release(R resource); +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java new file mode 100644 index 000000000000..53f3991b116d --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java @@ -0,0 +1,135 @@ +/* + * 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.pulsar.cache; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +public class LRUCacheTest { + + @Mock + private ResourcePool mockedResourcePool; + + @Mock + private PulsarProducer mockedPulsarProducer; + + @SuppressWarnings("unchecked") + @Before + public void setUp() throws InterruptedException { + mockedResourcePool = mock(ResourcePool.class); + mockedPulsarProducer = mock(PulsarProducer.class); + + when(mockedResourcePool.acquire(any(Properties.class))).thenReturn(mockedPulsarProducer); + } + + /** + * Make sure the LRUCache functions as a Map + */ + @Test + public void simpleTest() { + LRUCache cache = new LRUCache (10, mockedResourcePool); + + for(Character i='A'; i<='E'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + assertEquals(5, cache.getSize()); + + for(Character i='A'; i<='E'; i++){ + assertNotNull( cache.get(i.toString())); + } + } + + @Test + public void evictionTest() { + + LRUCache cache = new LRUCache (5, mockedResourcePool); + + for(Character i='A'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + // Make sure we only have 5 items in the cache + assertEquals(5, cache.getSize()); + + // Make sure we have the last 5 items added to the cache + for(Character i='V'; i<='Z'; i++){ + assertNotNull( cache.get(i.toString())); + } + + // Make sure the evict method on the resource pool was called 21 times, once for every item removed. + verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); + } + + @Test + public void evictionLruTest() { + + LRUCache cache = new LRUCache (5, mockedResourcePool); + + final Character A = 'A'; + + // Write 25 items to the cache, and the letter 'A' every other put. + for(Character i='B'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + cache.put(A.toString(), mockedPulsarProducer); + } + + // Make sure we only have 5 items in the cache + assertEquals(5, cache.getSize()); + + // Make sure that the letter 'A' is still in the cache due to frequent access + assertNotNull( cache.get(A.toString()) ); + + // Make sure we have the last 4 items added to the cache + for(Character i='W'; i<='Z'; i++){ + assertNotNull( cache.get(i.toString())); + } + + // Make sure the evict method on the resource pool was called 21 times, once for every item removed. + verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); + } + + @Test + public void clearTest() { + LRUCache cache = new LRUCache (26, mockedResourcePool); + + for(Character i='A'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + // Make sure we only have all the items in the cache + assertEquals(26, cache.getSize()); + + cache.clear(); + + // Make sure all the items were removed + assertEquals(0, cache.getSize()); + + // Make sure all the items were evicted from the underlying resource pool + verify(mockedResourcePool, times(26)).evict(mockedPulsarProducer); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml new file mode 100644 index 000000000000..cb3c116e3772 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml @@ -0,0 +1,47 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service-nar + 1.5.0 + nar + + true + true + + + + + org.apache.nifi + nifi-pulsar-client-service-api-nar + 1.5.0 + nar + + + org.apache.nifi + nifi-pulsar-client-service + 1.5.0 + + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml new file mode 100644 index 000000000000..366b9e66d4b8 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml @@ -0,0 +1,64 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + + + nifi-pulsar-client-service + jar + + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.5.0 + + + org.apache.nifi + nifi-api + provided + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-ssl-context-service-api + provided + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java new file mode 100644 index 000000000000..e965d99b89f0 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java @@ -0,0 +1,300 @@ +/* + * 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.pulsar; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.nifi.pulsar.pool.ResourcePoolImpl; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.pulsar.client.api.ClientConfiguration; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; + +@Tags({ "Pulsar"}) +@CapabilityDescription("Standard ControllerService implementation of PulsarClientService.") +public class StandardPulsarClientPool extends AbstractControllerService implements PulsarClientPool { + + public static final PropertyDescriptor PULSAR_SERVICE_URL = new PropertyDescriptor + .Builder().name("PULSAR_SERVICE_URL") + .displayName("Pulsar Service URL") + .description("URL for the Pulsar cluster, e.g localhost:6650") + .required(true) + .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR) + .build(); + + public static final PropertyDescriptor CONCURRENT_LOOKUP_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum concurrent lookup-requests") + .description("Number of concurrent lookup-requests allowed on each broker-connection to prevent " + + "overload on broker. (default: 5000) It should be configured with higher value only in case " + + "of it requires to produce/subscribe on thousands of topics") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("5000") + .build(); + + public static final PropertyDescriptor CONNECTIONS_PER_BROKER = new PropertyDescriptor.Builder() + .name("Maximum connects per Pulsar broker") + .description("Sets the max number of connection that the client library will open to a single broker.\n" + + "By default, the connection pool will use a single connection for all the producers and consumers. " + + "Increasing this parameter may improve throughput when using many producers over a high latency connection") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor IO_THREADS = new PropertyDescriptor.Builder() + .name("I/O Threads") + .description("The number of threads to be used for handling connections to brokers (default: 1 thread)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor LISTENER_THREADS = new PropertyDescriptor.Builder() + .name("Listener Threads") + .description("The number of threads to be used for message listeners (default: 1 thread)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAXIMUM_REJECTED_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum rejected requests per connection") + .description("Max number of broker-rejected requests in a certain time-frame (30 seconds) after " + + "which current connection will be closed and client creates a new connection that give " + + "chance to connect a different broker (default: 50)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("50") + .build(); + + public static final PropertyDescriptor OPERATION_TIMEOUT = new PropertyDescriptor.Builder() + .name("Operation Timeout") + .description("Producer-create, subscribe and unsubscribe operations will be retried until this " + + "interval, after which the operation will be maked as failed (default: 30 seconds)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("30") + .build(); + + public static final PropertyDescriptor STATS_INTERVAL = new PropertyDescriptor.Builder() + .name("Stats interval") + .description("The interval between each stat info (default: 60 seconds) Stats will be activated " + + "with positive statsIntervalSeconds It should be set to at least 1 second") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("60") + .build(); + + public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder() + .name("Use TCP nodelay flag") + .description("Configure whether to use TCP no-delay flag on the connection, to disable Nagle algorithm.\n" + + "No-delay features make sure packets are sent out on the network as soon as possible, and it's critical " + + "to achieve low latency publishes. On the other hand, sending out a huge number of small packets might " + + "limit the overall throughput, so if latency is not a concern, it's advisable to set the useTcpNoDelay " + + "flag to false.") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor MAX_PRODUCERS = new PropertyDescriptor + .Builder().name("MAX_PRODUCERS") + .displayName("Producer Pool Size") + .description("The Maximum Number of Pulsar Producers created by this Pulsar Client Pool") + .required(true) + .defaultValue("10") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final PropertyDescriptor MAX_CONSUMERS = new PropertyDescriptor + .Builder().name("MAX_CONSUMERS") + .displayName("Consumer Pool Size") + .description("The Maximum Number of Pulsar consumers created by this Pulsar Client Pool") + .required(true) + .defaultValue("10") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("ssl.context.service") + .displayName("SSL Context Service") + .description("Specifies the SSL Context Service to use for communicating with Pulsar.") + .required(false) + .identifiesControllerService(SSLContextService.class) + .build(); + + private static final List properties; + private volatile PulsarClient client; + + private volatile ResourcePoolImpl producers; + private volatile ResourcePoolImpl consumers; + private ClientConfiguration clientConfig; + + static { + final List props = new ArrayList<>(); + props.add(PULSAR_SERVICE_URL); + props.add(MAX_CONSUMERS); + props.add(MAX_PRODUCERS); + props.add(CONCURRENT_LOOKUP_REQUESTS); + props.add(CONNECTIONS_PER_BROKER); + props.add(IO_THREADS); + props.add(LISTENER_THREADS); + props.add(MAXIMUM_REJECTED_REQUESTS); + props.add(OPERATION_TIMEOUT); + props.add(STATS_INTERVAL); + props.add(USE_TCP_NO_DELAY); + properties = Collections.unmodifiableList(props); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + /** + * @param context + * the configuration context + * @throws InitializationException + * if unable to create a database connection + */ + @OnEnabled + public void onEnabled(final ConfigurationContext context) throws InitializationException { + + createClient(context); + + if (this.client == null) + throw new InitializationException("Unable to create Pulsar Client"); + + producers = new ResourcePoolImpl(new PulsarProducerFactory(client), context.getProperty(MAX_PRODUCERS).asInteger()); + consumers = new ResourcePoolImpl(new PulsarConsumerFactory(client), context.getProperty(MAX_CONSUMERS).asInteger()); + + } + + private void createClient(final ConfigurationContext context) throws InitializationException { + + // We can't create a client without a service URL. + if (!context.getProperty(PULSAR_SERVICE_URL).isSet()) { + return; + } + + try { + this.client = PulsarClient.create(buildPulsarBrokerRootUrl(context.getProperty(PULSAR_SERVICE_URL).getValue(), + getClientConfig(context).isUseTls()), getClientConfig(context)); + + } catch (Exception e) { + throw new InitializationException("Unable to create Pulsar Client", e); + } + + } + + private static String buildPulsarBrokerRootUrl(String uri, boolean tlsEnabled) { + StringBuilder builder = new StringBuilder(); + builder.append("pulsar"); + + if (tlsEnabled) + builder.append("+ssl"); + + builder.append("://"); + builder.append(uri); + return builder.toString(); + } + + private ClientConfiguration getClientConfig(ConfigurationContext context) throws UnsupportedAuthenticationException { + + if (clientConfig == null) { + clientConfig = new ClientConfiguration(); + + if (context.getProperty(CONCURRENT_LOOKUP_REQUESTS).isSet()) { + clientConfig.setConcurrentLookupRequest(context.getProperty(CONCURRENT_LOOKUP_REQUESTS).asInteger()); + } + + if (context.getProperty(CONNECTIONS_PER_BROKER).isSet()) { + clientConfig.setConnectionsPerBroker(context.getProperty(CONNECTIONS_PER_BROKER).asInteger()); + } + + if (context.getProperty(IO_THREADS).isSet()) { + clientConfig.setIoThreads(context.getProperty(IO_THREADS).asInteger()); + } + + if (context.getProperty(LISTENER_THREADS).isSet()) { + clientConfig.setListenerThreads(context.getProperty(LISTENER_THREADS).asInteger()); + } + + if (context.getProperty(MAXIMUM_REJECTED_REQUESTS).isSet()) { + clientConfig.setMaxNumberOfRejectedRequestPerConnection(context.getProperty(MAXIMUM_REJECTED_REQUESTS).asInteger()); + } + + if (context.getProperty(OPERATION_TIMEOUT).isSet()) { + clientConfig.setOperationTimeout(context.getProperty(OPERATION_TIMEOUT).asInteger(), TimeUnit.SECONDS); + } + + if (context.getProperty(STATS_INTERVAL).isSet()) { + clientConfig.setStatsInterval(context.getProperty(STATS_INTERVAL).asLong(), TimeUnit.SECONDS); + } + + if (context.getProperty(USE_TCP_NO_DELAY).isSet()) { + clientConfig.setUseTcpNoDelay(context.getProperty(USE_TCP_NO_DELAY).asBoolean()); + } + + // Configure TLS + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + + if (sslContextService != null && sslContextService.isTrustStoreConfigured() && sslContextService.isKeyStoreConfigured()) { + clientConfig.setUseTls(true); + clientConfig.setTlsTrustCertsFilePath(sslContextService.getTrustStoreFile()); + + Map authParams = new HashMap<>(); + + // TODO This should be a different value than the TlsTrustCertsFilePath above. + authParams.put("tlsCertFile", sslContextService.getTrustStoreFile()); + authParams.put("tlsKeyFile", sslContextService.getKeyStoreFile()); + clientConfig.setAuthentication(AuthenticationTls.class.getName(), authParams); + } + } + + return clientConfig; + } + + @Override + public ResourcePool getProducerPool() { + return this.producers; + } + + @Override + public ResourcePool getConsumerPool() { + return this.consumers; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java new file mode 100644 index 000000000000..86d4934278d2 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java @@ -0,0 +1,31 @@ +/* + * 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.pulsar.pool; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ResourceExceptionHandlerImpl implements ResourceExceptionHandler { + + private static Logger logger = LoggerFactory.getLogger(ResourceExceptionHandlerImpl.class); + + @Override + public void handle(Exception exc) { + logger.error("Unable to create Resource", exc); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java new file mode 100644 index 000000000000..90ddb95aca13 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java @@ -0,0 +1,139 @@ +/* + * 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.pulsar.pool; + +import java.util.Iterator; +import java.util.Properties; +import java.util.Vector; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + + +public class ResourcePoolImpl implements ResourcePool { + + private final Lock lock = new ReentrantLock(); + private final Condition poolAvailable = lock.newCondition(); + private int max_resources; + private final Vector pool; + + private final ResourceExceptionHandler resourceExceptionHandler; + private final ResourceFactory factory; + + public ResourcePoolImpl(ResourceFactory factory, int max_resources) { + this(factory, new ResourceExceptionHandlerImpl(), max_resources); + } + + public ResourcePoolImpl(ResourceFactory factory, ResourceExceptionHandler handler, int max_resources) { + lock.lock(); + try { + this.factory = factory; + this.resourceExceptionHandler = handler; + this.max_resources = max_resources; + this.pool = new Vector (max_resources); + } finally { + lock.unlock(); + } + } + + private R createResource(Properties props) { + R resource = null; + try { + + resource = factory.create(props); + + if (resource == null) + throw new ResourceCreationException("Unable to create resource"); + + } catch (Exception e) { + resourceExceptionHandler.handle(e); + } + return resource; + } + + + /* + * Shutdown the pool and release the resources + */ + public void close() { + + Iterator itr = pool.iterator(); + while (itr.hasNext()) { + itr.next().close(); + } + + } + + public boolean isEmpty() { + return (pool.isEmpty()); + } + + public boolean isFull() { + return (pool != null && pool.size() == max_resources); + } + + @Override + public R acquire(Properties props) throws InterruptedException { + lock.lock(); + try { + while (max_resources <= 0) { + poolAvailable.await(); + } + + + --max_resources; + + if (pool != null) { + int size = pool.size(); + if (size > 0) + return pool.remove(size - 1); + } + return createResource(props); + } finally { + lock.unlock(); + } + } + + @Override + public void evict(R resource) { + lock.lock(); + try { + + // Attempt to close the connection + if (!resource.isClosed()) + resource.close(); + + pool.removeElement(resource); + --max_resources; + poolAvailable.signal(); + } finally { + lock.unlock(); + } + } + + @Override + public void release(R resource) { + lock.lock(); + try { + pool.addElement(resource); + ++max_resources; + poolAvailable.signal(); + } finally { + lock.unlock(); + } + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 000000000000..c1eecc70a562 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,15 @@ +# 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. +org.apache.nifi.pulsar.StandardPulsarClientPool \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java new file mode 100644 index 000000000000..72d17006e83f --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java @@ -0,0 +1,45 @@ +/* + * 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.pulsar; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +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; + +public class TestProcessor extends AbstractProcessor { + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + } + + @Override + protected List getSupportedPropertyDescriptors() { + List propDescs = new ArrayList<>(); + propDescs.add(new PropertyDescriptor.Builder() + .name("StandardPulsarClientService test processor") + .description("StandardPulsarClientService test processor") + .identifiesControllerService(StandardPulsarClientPool.class) + .required(true) + .build()); + return propDescs; + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java new file mode 100644 index 000000000000..4b9982df30db --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java @@ -0,0 +1,44 @@ +/* + * 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.pulsar; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class TestStandardPulsarClientService { + + @Before + public void init() { + + } + + @Test + public void testService() throws InitializationException { + final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class); + final PulsarClientPool service = new StandardPulsarClientPool(); + runner.addControllerService("test-good", service); + + runner.setProperty(service, StandardPulsarClientPool.PULSAR_SERVICE_URL, "localhost:6667"); + // runner.enableControllerService(service); + + runner.assertValid(service); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java new file mode 100644 index 000000000000..a19fbad556d3 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java @@ -0,0 +1,146 @@ +/* + * 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.pulsar.pool; + +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Rule; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.stubbing.Answer; + +import static org.mockito.Mockito.*; + +import java.util.Properties; + +public class MockPulsarClientService extends AbstractControllerService implements PulsarClientPool { + + @Mock + PulsarClient mockClient; + + @Mock + ResourcePool mockProducerPool; + + @Mock + ResourcePool mockConsumerPool; + + @Mock + Producer mockProducer; + + @Mock + Consumer mockConsumer; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @SuppressWarnings("unchecked") + public MockPulsarClientService() { + this.mockClient = mock(PulsarClient.class); + mockProducerPool = mock(ResourcePool.class); + mockConsumerPool = mock(ResourcePool.class); + mockProducer = mock(Producer.class); + mockConsumer = mock(Consumer.class); + + try { + when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarProducer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getProducer(props); + } + } + ); + + when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( + new Answer () { + @Override + public PulsarConsumer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getConsumer(props); + } + } + ); + } catch (InterruptedException ex) { + + } + + try { + when(mockClient.createProducer(anyString())).thenReturn(getMockProducer()); + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(null); + } catch (PulsarClientException e) { + e.printStackTrace(); + } + } + + + public Producer getMockProducer() { + return mockProducer; + } + + public PulsarClient getMockClient() { + return mockClient; + } + + public PulsarProducer getProducer(Properties props) { + String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); + try { + return new PulsarProducer(mockClient.createProducer(topic), topic); + } catch (PulsarClientException e) { + return null; + } + } + + public PulsarConsumer getConsumer(Properties props) { + String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); + String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); + try { + return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + } catch (PulsarClientException e) { + return null; + } + } + + @Override + public ResourcePool getProducerPool() { + return mockProducerPool; + } + + + @Override + public ResourcePool getConsumerPool() { + return mockConsumerPool; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java new file mode 100644 index 000000000000..397b53880351 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java @@ -0,0 +1,146 @@ +/* + * 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.pulsar.pool; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.junit.Test; + +public class ResourcePoolTest { + + private MockPulsarClientService pulsarClient = new MockPulsarClientService(); + + /*Checks the number of the resources in the Resource Table. + Resource is created only if needed.*/ + @Test + public void testIsEmptyBeforeResourceAcquired() throws Exception { + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + assertTrue(resPool.isEmpty()); + } + + /* Checks if the number of resources in the Resource Table is equal to the + * maximum number of resources declared at the Resource Pool creation time.*/ + @Test + public void testIsFull() throws Exception { + + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + } + + /* + * Checks to see if resources are left in the pool after they + * have been acquired, and then released. + */ + @Test + public void testResourcesReused() throws InterruptedException { + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + for (int i = 0; i <= 4; i++) { + resPool.release(res[i]); + } + + assertFalse(resPool.isEmpty()); + } + + /* + * Checks to see if resources that are invalidated are removed from + * the pool. + */ + @Test + public void testResourcesReleased() throws InterruptedException { + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + for (int i = 0; i <= 4; i++) { + resPool.evict(res[i]); + } + + assertTrue(resPool.isEmpty()); + } + + @Test + public void testAcquireBlocksWhenEmpty() throws Exception { + + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "A"); + + final ResourcePoolImpl resPool = new ResourcePoolImpl( + new PulsarProducerFactory(pulsarClient.getMockClient()) , 0); + + Thread resourceConsumer = new Thread() { + public void run() { + try { + @SuppressWarnings("unused") + PulsarProducer unused = resPool.acquire(props); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + fail(); // error if control flow reaches this line + } + }; + + resourceConsumer.start(); + Thread.sleep(1000); // waits for the resourceConsumer to block + resourceConsumer.interrupt(); + resourceConsumer.join(1000); // resume after the resourceConsumer ends + assertFalse(resourceConsumer.isAlive()); + + } + + + private ResourcePoolImpl getResourcePool(String topic, int size) { + return new ResourcePoolImpl( + new PulsarProducerFactory(pulsarClient.getMockClient()), size); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml new file mode 100644 index 000000000000..680d08a13457 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml @@ -0,0 +1,37 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 1.5.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.5.0 + pom + + + nifi-pulsar-client-service-api + nifi-pulsar-client-service-api-nar + nifi-pulsar-client-service + nifi-pulsar-client-service-nar + + + From 29ca7f2b653503de9ad13da9f6c49cc2f311eaae Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Mon, 26 Feb 2018 11:19:26 -0800 Subject: [PATCH 05/16] Updated all nifi component version references to 1.6.0-SNAPSHOT --- .../nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml | 6 +++--- .../nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml | 4 ++-- nifi-nar-bundles/nifi-pulsar-bundle/pom.xml | 4 ++-- .../nifi-pulsar-client-service-api-nar/pom.xml | 6 +++--- .../nifi-pulsar-client-service-api/pom.xml | 2 +- .../nifi-pulsar-client-service-nar/pom.xml | 8 ++++---- .../nifi-pulsar-client-service/pom.xml | 4 ++-- nifi-nar-bundles/nifi-pulsar-client-services/pom.xml | 4 ++-- 8 files changed, 19 insertions(+), 19 deletions(-) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml index ca78a838211f..d986bf50fb98 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-bundle - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-nar @@ -33,13 +33,13 @@ org.apache.nifi nifi-pulsar-processors - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi nifi-pulsar-client-service-api-nar - 1.5.0 + 1.6.0-SNAPSHOT nar diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml index 5ebafd05e6fa..2931a50e918f 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-bundle - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-processors @@ -49,7 +49,7 @@ org.apache.nifi nifi-pulsar-client-service-api - 1.5.0 + 1.6.0-SNAPSHOT provided diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml index ccb90f23350b..abb745eb259f 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml @@ -19,12 +19,12 @@ org.apache.nifi nifi-nar-bundles - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi nifi-pulsar-bundle - 1.5.0 + 1.6.0-SNAPSHOT pom diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml index a0bd2122c2ae..4e6ed6d98a03 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml @@ -19,11 +19,11 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service-api-nar - 1.5.0 + 1.6.0-SNAPSHOT nar true @@ -39,7 +39,7 @@ org.apache.nifi nifi-pulsar-client-service-api - 1.5.0 + 1.6.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml index 2218c3258649..2259fc94f5ff 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service-api diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml index cb3c116e3772..460e2e12febf 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml @@ -19,11 +19,11 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service-nar - 1.5.0 + 1.6.0-SNAPSHOT nar true @@ -34,13 +34,13 @@ org.apache.nifi nifi-pulsar-client-service-api-nar - 1.5.0 + 1.6.0-SNAPSHOT nar org.apache.nifi nifi-pulsar-client-service - 1.5.0 + 1.6.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml index 366b9e66d4b8..5c3d3a6efc49 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT nifi-pulsar-client-service @@ -29,7 +29,7 @@ org.apache.nifi nifi-pulsar-client-service-api - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml index 680d08a13457..37b8e3fa3e54 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml @@ -19,12 +19,12 @@ org.apache.nifi nifi-nar-bundles - 1.5.0 + 1.6.0-SNAPSHOT org.apache.nifi nifi-pulsar-client-services - 1.5.0 + 1.6.0-SNAPSHOT pom From 3e30c9530ab0cd844fb900607dd00ff1a4014221 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Wed, 28 Feb 2018 10:18:35 -0800 Subject: [PATCH 06/16] Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 --- ...sumePulsar.java => ConsumePulsar_1_0.java} | 2 +- ...lishPulsar.java => PublishPulsar_1_0.java} | 2 +- .../org.apache.nifi.processor.Processor | 4 +- .../pulsar/AbstractPulsarProcessorTest.java | 2 +- .../pulsar/ConsumePulsarProcessorTest.java | 24 +++++----- .../pulsar/PublishPulsarProcessorTest.java | 44 +++++++++---------- .../apache/nifi/pulsar/PulsarClientPool.java | 14 ------ 7 files changed, 39 insertions(+), 53 deletions(-) rename nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/{ConsumePulsar.java => ConsumePulsar_1_0.java} (99%) rename nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/{PublishPulsar.java => PublishPulsar_1_0.java} (99%) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java similarity index 99% rename from nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java rename to nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java index b84cb7aa70c5..f331b11d4944 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java @@ -58,7 +58,7 @@ @CapabilityDescription("Consumes messages from Apache Pulsar " + "The complementary NiFi processor for sending messages is PublishPulsar.") @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) -public class ConsumePulsar extends AbstractPulsarProcessor { +public class ConsumePulsar_1_0 extends AbstractPulsarProcessor { static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java similarity index 99% rename from nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java rename to nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java index 20b93ddfd234..028d7061d00d 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java @@ -63,7 +63,7 @@ @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " + "FlowFiles that are routed to success.") -public class PublishPulsar extends AbstractPulsarProcessor { +public class PublishPulsar_1_0 extends AbstractPulsarProcessor { protected static final String MSG_COUNT = "msg.count"; diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index 93e412f3fc13..fbab5769250f 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -12,5 +12,5 @@ # 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. -org.apache.nifi.processors.pulsar.ConsumePulsar -org.apache.nifi.processors.pulsar.PublishPulsar \ No newline at end of file +org.apache.nifi.processors.pulsar.ConsumePulsar_1_0 +org.apache.nifi.processors.pulsar.PublishPulsar_1_0 \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java index c11008209f63..1bee9dc120a5 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -32,6 +32,6 @@ protected void addPulsarClientService() throws InitializationException { final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); runner.addControllerService("pulsarClient", pulsarClient); runner.enableControllerService(pulsarClient); - runner.setProperty(PublishPulsar.PULSAR_CLIENT_SERVICE, "pulsarClient"); + runner.setProperty(PublishPulsar_1_0.PULSAR_CLIENT_SERVICE, "pulsarClient"); } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java index 64550d442043..84ae6a4a56e2 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -49,7 +49,7 @@ public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { - runner = TestRunners.newTestRunner(ConsumePulsar.class); + runner = TestRunners.newTestRunner(ConsumePulsar_1_0.class); mockClient = mock(PulsarClient.class); mockConsumer = mock(Consumer.class); @@ -77,10 +77,10 @@ public void init() throws InitializationException { public void emptyMessageTest() { when(mockMessage.getData()).thenReturn("".getBytes()); - runner.setProperty(ConsumePulsar.TOPIC, "foo"); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); runner.run(); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); } @Test @@ -110,10 +110,10 @@ public void multipleAsyncMessagesTest() throws PulsarClientException { public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); - runner.setProperty(ConsumePulsar.TOPIC, "foo"); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); runner.run(10, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); runner.assertQueueEmpty(); @@ -132,14 +132,14 @@ private void sendMessages(String msg, String topic, String sub, boolean async, i when(mockMessage.getData()).thenReturn(msg.getBytes()); - runner.setProperty(ConsumePulsar.ASYNC_ENABLED, Boolean.toString(async)); - runner.setProperty(ConsumePulsar.TOPIC, topic); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, sub); + runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); runner.run(itertions, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); - List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar.REL_SUCCESS); + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_0.REL_SUCCESS); assertEquals(itertions, flowFiles.size()); for (MockFlowFile ff : flowFiles) { diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java index 30717a649e68..0b15b0ae5ec8 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -44,7 +44,7 @@ public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { - runner = TestRunners.newTestRunner(PublishPulsar.class); + runner = TestRunners.newTestRunner(PublishPulsar_1_0.class); mockClient = mock(PulsarClient.class); mockProducer = mock(Producer.class); @@ -82,15 +82,15 @@ public boolean matches(Object argument) { @Test public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); final String content = "some content"; Map attributes = new HashMap (); - attributes.put(PublishPulsar.TOPIC.getName(), ""); + attributes.put(PublishPulsar_1_0.TOPIC.getName(), ""); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_FAILURE); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); // Confirm that no Producer as created verify(mockClient, times(0)).createProducer(anyString()); @@ -99,15 +99,15 @@ public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); final String content = "some content"; Map attributes = new HashMap (); - attributes.put(PublishPulsar.TOPIC.getName(), "topic-b"); + attributes.put(PublishPulsar_1_0.TOPIC.getName(), "topic-b"); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); // Verify that we sent the data to topic-b. verify(mockClient, times(1)).createProducer("topic-b"); @@ -116,14 +116,14 @@ public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -136,15 +136,15 @@ public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClie @Test public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -157,14 +157,14 @@ public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, Pulsa @Test public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); } // Verify that the send method on the producer was called with the expected content @@ -174,15 +174,15 @@ public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarC @Test public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); } // Verify that the send method on the producer was called with the expected content @@ -191,14 +191,14 @@ public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, Pu @Test public void stressTest() throws UnsupportedEncodingException { - runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; for (int idx = 0; idx < 9999; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); } } diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java index be57a42a0d8c..be619eae4297 100644 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java +++ b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java @@ -27,20 +27,6 @@ + "properties defined") public interface PulsarClientPool extends ControllerService { - /* - public PulsarProducer getProducer(Properties properties) throws PulsarClientException; - - public void release(PulsarProducer producer); - - public void invalidate(PulsarProducer producer); - - public PulsarConsumer getConsumer(Properties properties) throws PulsarClientException; - - public void release(PulsarConsumer consumer); - - public void invalidate(PulsarConsumer consumer); -*/ - public ResourcePool getProducerPool(); public ResourcePool getConsumerPool(); From 20b451d66ba6f7bbcc057f752a25101e33580469 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 23 Feb 2018 17:40:22 -0800 Subject: [PATCH 07/16] Added Pulsar processors and Controller Service --- .../nifi/processors/pulsar/ConsumePulsar.java | 391 ++++++++++++++++++ .../nifi/processors/pulsar/PublishPulsar.java | 372 +++++++++++++++++ .../org.apache.nifi.processor.Processor | 7 +- .../pulsar/AbstractPulsarProcessorTest.java | 4 + .../pulsar/ConsumePulsarProcessorTest.java | 29 ++ .../pulsar/PublishPulsarProcessorTest.java | 75 ++++ .../nifi-pulsar-client-services/.gitignore | 5 - .../pom.xml | 46 --- .../nifi-pulsar-client-service-api/pom.xml | 40 -- .../apache/nifi/pulsar/PulsarClientPool.java | 33 -- .../apache/nifi/pulsar/PulsarConsumer.java | 70 ---- .../apache/nifi/pulsar/PulsarProducer.java | 64 --- .../apache/nifi/pulsar/cache/LRUCache.java | 69 ---- .../nifi/pulsar/pool/PoolableResource.java | 25 -- .../pulsar/pool/PulsarConsumerFactory.java | 56 --- .../pulsar/pool/PulsarProducerFactory.java | 53 --- .../pool/ResourceCreationException.java | 31 -- .../pulsar/pool/ResourceExceptionHandler.java | 23 -- .../nifi/pulsar/pool/ResourceFactory.java | 24 -- .../apache/nifi/pulsar/pool/ResourcePool.java | 43 -- .../nifi/pulsar/cache/LRUCacheTest.java | 135 ------ .../nifi-pulsar-client-service-nar/pom.xml | 47 --- .../nifi-pulsar-client-service/pom.xml | 64 --- .../nifi/pulsar/StandardPulsarClientPool.java | 300 -------------- .../pool/ResourceExceptionHandlerImpl.java | 31 -- .../nifi/pulsar/pool/ResourcePoolImpl.java | 139 ------- ...g.apache.nifi.controller.ControllerService | 15 - .../org/apache/nifi/pulsar/TestProcessor.java | 45 -- .../TestStandardPulsarClientService.java | 44 -- .../pulsar/pool/MockPulsarClientService.java | 146 ------- .../nifi/pulsar/pool/ResourcePoolTest.java | 146 ------- .../nifi-pulsar-client-services/pom.xml | 37 -- 32 files changed, 877 insertions(+), 1732 deletions(-) create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/.gitignore delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-client-services/pom.xml diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java new file mode 100644 index 000000000000..b84cb7aa70c5 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerConfiguration; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionType; + +@Tags({"Pulsar", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume"}) +@CapabilityDescription("Consumes messages from Apache Pulsar " + + "The complementary NiFi processor for sending messages is PublishPulsar.") +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +public class ConsumePulsar extends AbstractPulsarProcessor { + + static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); + static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); + static final AllowableValue FAILOVER = new AllowableValue("Failover", "Failover", "Multiple consumer will be able to use the same subscription name but only 1 consumer " + + "will receive the messages. If that consumer disconnects, one of the other connected consumers will start receiving messages"); + + protected static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + static final PropertyDescriptor SUBSCRIPTION = new PropertyDescriptor.Builder() + .name("Subscription") + .displayName("Subscription Name") + .description("The name of the Pulsar subscription to consume from.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + .name("Async Enabled") + .description("Control whether the messages will be consumed asyncronously or not. Messages consumed" + + " syncronously will be acknowledged immediately before processing the next message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum Async Requests") + .description("The maximum number of outstanding asynchronous consumer requests for this processor. " + + "Each asynchronous call requires memory, so avoid setting this value to high.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("50") + .build(); + + public static final PropertyDescriptor ACK_TIMEOUT = new PropertyDescriptor.Builder() + .name("Acknowledgment Timeout") + .description("Set the timeout (in milliseconds) for unacked messages, truncated to the " + + "nearest millisecond. The timeout needs to be greater than 10 seconds.") + .required(false) + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .defaultValue("10000") + .build(); + + public static final PropertyDescriptor PRIORITY_LEVEL = new PropertyDescriptor.Builder() + .name("Consumer Priority Level") + .description("Sets priority level for the shared subscription consumers to which broker " + + "gives more priority while dispatching messages. Here, broker follows descending " + + "priorities. (eg: 0=max-priority, 1, 2,..) ") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("5") + .build(); + + public static final PropertyDescriptor RECEIVER_QUEUE_SIZE = new PropertyDescriptor.Builder() + .name("Consumer receive queue size.") + .description("The consumer receive queue controls how many messages can be accumulated " + + "by the Consumer before the application calls Consumer.receive(). Using a higher " + + "value could potentially increase the consumer throughput at the expense of bigger " + + "memory utilization. \n" + + "Setting the consumer queue size as zero, \n" + + "\t - Decreases the throughput of the consumer, by disabling pre-fetching of messages. \n" + + "\t - Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer " + + "connection with broker and consumer will not be able receive any further message unless batch-message " + + "in pipeline is removed") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + public static final PropertyDescriptor SUBSCRIPTION_TYPE = new PropertyDescriptor.Builder() + .name("Subscription Type") + .description("Select the subscription type to be used when subscribing to the topic.") + .required(false) + .allowableValues(EXCLUSIVE, SHARED, FAILOVER) + .defaultValue(SHARED.getValue()) + .build(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + // Reuse the same consumer for a given topic / subscription + private PulsarConsumer consumer; + private ConsumerConfiguration consumerConfig; + + // Pool for running multiple consume Async requests + ExecutorService pool; + ExecutorCompletionService completionService; + + static { + final List properties = new ArrayList<>(); + properties.add(PULSAR_CLIENT_SERVICE); + properties.add(TOPIC); + properties.add(SUBSCRIPTION); + properties.add(ASYNC_ENABLED); + properties.add(MAX_ASYNC_REQUESTS); + properties.add(ACK_TIMEOUT); + properties.add(PRIORITY_LEVEL); + properties.add(RECEIVER_QUEUE_SIZE); + properties.add(SUBSCRIPTION_TYPE); + + PROPERTIES = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + RELATIONSHIPS = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTIES; + } + + @OnScheduled + public void init(ProcessContext context) { + pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); + completionService = new ExecutorCompletionService<>(pool); + } + + @OnUnscheduled + public void shutDown() { + // Stop all the async consumers + pool.shutdownNow(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + try { + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + // Launch consumers + consumeAsync(context, session); + + // Handle completed consumers + handleAsync(context, session); + + } else { + consume(context, session); + } + } catch (PulsarClientException e) { + getLogger().error("Unable to consume from Pulsar Topic ", e); + context.yield(); + throw new ProcessException(e); + } + + } + + private void handleAsync(ProcessContext context, ProcessSession session) { + + try { + Future done = completionService.take(); + Message msg = done.get(); + + if (msg != null) { + FlowFile flowFile = null; + final byte[] value = msg.getData(); + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + } + + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.transfer(flowFile, REL_SUCCESS); + session.commit(); + getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); + } + + } catch (InterruptedException | ExecutionException | PulsarClientException e) { + getLogger().error("Trouble consuming messages ", e); + } + + } + + @OnStopped + public void close(final ProcessContext context) { + + getLogger().info("Disconnecting Pulsar Consumer"); + if (consumer != null) { + + context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getConsumerPool().evict(consumer); + } + + consumer = null; + } + + /* + * For now let's assume that this processor will be configured to run for a longer + * duration than 0 milliseconds. So we will be grabbing as many messages off the topic + * as possible and committing them as FlowFiles + */ + private void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException { + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + completionService.submit(new Callable() { + @Override + public Message call() throws Exception { + return consumer.receiveAsync().get(); + } + }); + + } + + /* + * When this Processor expects to receive many small files, it may + * be advisable to create several FlowFiles from a single session + * before committing the session. Typically, this allows the Framework + * to treat the content of the newly created FlowFiles much more efficiently. + */ + private void consume(ProcessContext context, ProcessSession session) throws PulsarClientException { + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + final ComponentLog logger = getLogger(); + final Message msg; + FlowFile flowFile = null; + + try { + + msg = consumer.receive(); + final byte[] value = msg.getData(); + + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.transfer(flowFile, REL_SUCCESS); + logger.info("Created {} from {} messages received from Pulsar Server and transferred to 'success'", + new Object[]{flowFile, 1}); + + session.commit(); + + /* + * This Processor acknowledges receipt of the data and/or removes the data + * from the external source in order to prevent receipt of duplicate files. + * This is done only after the ProcessSession by which the FlowFile was created + * has been committed! Failure to adhere to this principle may result in data + * loss, as restarting NiFi before the session has been committed will result + * in the temporary file being deleted. Note, however, that it is possible using + * this approach to receive duplicate data because the application could be + * restarted after committing the session and before acknowledging or removing + * the data from the external source. In general, though, potential data duplication + * is preferred over potential data loss. + */ + getLogger().info("Acknowledging message " + msg.getMessageId()); + consumer.acknowledge(msg); + + } else { + // We didn't consume any data, so + session.commit(); + } + + } catch (PulsarClientException e) { + context.yield(); + session.rollback(); + } + + } + + private PulsarConsumer getWrappedConsumer(ProcessContext context) throws PulsarClientException { + + if (consumer != null) + return consumer; + + final PulsarClientPool pulsarClientService = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class); + + try { + consumer = pulsarClientService.getConsumerPool() + .acquire(getConsumerProperties(context)); + + if (consumer == null || consumer.getConsumer() == null) { + throw new PulsarClientException("Unable to create Pulsar Consumer"); + } + + return consumer; + } catch (final InterruptedException ex) { + return null; + } + } + + private Properties getConsumerProperties(ProcessContext context) { + + Properties props = new Properties(); + props.put(PulsarConsumerFactory.TOPIC_NAME, context.getProperty(TOPIC).getValue()); + props.put(PulsarConsumerFactory.SUBSCRIPTION_NAME, context.getProperty(SUBSCRIPTION).getValue()); + props.put(PulsarConsumerFactory.CONSUMER_CONFIG, getConsumerConfig(context)); + return props; + } + + private ConsumerConfiguration getConsumerConfig(ProcessContext context) { + + if (consumerConfig == null) { + consumerConfig = new ConsumerConfiguration(); + + if (context.getProperty(ACK_TIMEOUT).isSet()) + consumerConfig.setAckTimeout(context.getProperty(ACK_TIMEOUT).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(PRIORITY_LEVEL).isSet()) + consumerConfig.setPriorityLevel(context.getProperty(PRIORITY_LEVEL).asInteger()); + + if (context.getProperty(RECEIVER_QUEUE_SIZE).isSet()) + consumerConfig.setReceiverQueueSize(context.getProperty(RECEIVER_QUEUE_SIZE).asInteger()); + + if (context.getProperty(SUBSCRIPTION_TYPE).isSet()) + consumerConfig.setSubscriptionType(SubscriptionType.valueOf(context.getProperty(SUBSCRIPTION_TYPE).getValue())); + } + + return consumerConfig; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java new file mode 100644 index 000000000000..20b93ddfd234 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.pulsar; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.cache.LRUCache; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.util.StringUtils; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConfiguration; +import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode; +import org.apache.pulsar.client.api.PulsarClientException; + +@Tags({"Apache", "Pulsar", "Put", "Send", "Message", "PubSub"}) +@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Pulsar using the Pulsar 1.21 Producer API." + + "The messages to send may be individual FlowFiles or may be delimited, using a " + + "user-specified delimiter, such as a new-line. " + + "The complementary NiFi processor for fetching messages is ConsumePulsar.") +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " + + "FlowFiles that are routed to success.") +public class PublishPulsar extends AbstractPulsarProcessor { + + protected static final String MSG_COUNT = "msg.count"; + + static final AllowableValue COMPRESSION_TYPE_NONE = new AllowableValue("NONE", "None", "No compression"); + static final AllowableValue COMPRESSION_TYPE_LZ4 = new AllowableValue("LZ4", "LZ4", "Compress with LZ4 algorithm."); + static final AllowableValue COMPRESSION_TYPE_ZLIB = new AllowableValue("ZLIB", "ZLIB", "Compress with ZLib algorithm"); + + static final AllowableValue MESSAGE_ROUTING_MODE_CUSTOM_PARTITION = new AllowableValue("CustomPartition", "Custom Partition", "Route messages to a custom partition"); + static final AllowableValue MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION = new AllowableValue("RoundRobinPartition", "Round Robin Partition", "Route messages to all partitions in a round robin manner"); + static final AllowableValue MESSAGE_ROUTING_MODE_SINGLE_PARTITION = new AllowableValue("SinglePartition", "Single Partition", "Route messages to a single partition"); + + public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + .name("Async Enabled") + .description("Control whether the messages will be sent asyncronously or not. Messages sent" + + " syncronously will be acknowledged immediately before processing the next message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor BATCHING_ENABLED = new PropertyDescriptor.Builder() + .name("Batching Enabled") + .description("Control whether automatic batching of messages is enabled for the producer. " + + "default: false [No batching] When batching is enabled, multiple calls to " + + "Producer.sendAsync can result in a single batch to be sent to the broker, leading " + + "to better throughput, especially when publishing small messages. If compression is " + + "enabled, messages will be compressed at the batch level, leading to a much better " + + "compression ratio for similar headers or contents. When enabled default batch delay " + + "is set to 10 ms and default batch size is 1000 messages") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor BATCHING_MAX_MESSAGES = new PropertyDescriptor.Builder() + .name("Batching Max Messages") + .description("Set the maximum number of messages permitted in a batch. default: " + + "1000 If set to a value greater than 1, messages will be queued until this " + + "threshold is reached or batch interval has elapsed") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + public static final PropertyDescriptor BATCH_INTERVAL = new PropertyDescriptor.Builder() + .name("Batch Interval") + .description("Set the time period within which the messages sent will be batched default: 10ms " + + "if batch messages are enabled. If set to a non zero value, messages will be queued until " + + "this time interval or until the Batching Max Messages threshould has been reached") + .required(false) + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .defaultValue("10") + .build(); + + public static final PropertyDescriptor BLOCK_IF_QUEUE_FULL = new PropertyDescriptor.Builder() + .name("Block if Message Queue Full") + .description("Set whether the processor should block when the outgoing message queue is full. " + + "Default is false. If set to false, send operations will immediately fail with " + + "ProducerQueueIsFullError when there is no space left in pending queue.") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() + .name("Compression Type") + .description("Set the compression type for the producer.") + .required(false) + .allowableValues(COMPRESSION_TYPE_NONE, COMPRESSION_TYPE_LZ4, COMPRESSION_TYPE_ZLIB) + .defaultValue(COMPRESSION_TYPE_NONE.getValue()) + .build(); + + public static final PropertyDescriptor MESSAGE_ROUTING_MODE = new PropertyDescriptor.Builder() + .name("Message Routing Mode") + .description("Set the message routing mode for the producer. This applies only if the destination topic is partitioned") + .required(false) + .allowableValues(MESSAGE_ROUTING_MODE_CUSTOM_PARTITION, MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION, MESSAGE_ROUTING_MODE_SINGLE_PARTITION) + .defaultValue(MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION.getValue()) + .build(); + + public static final PropertyDescriptor PENDING_MAX_MESSAGES = new PropertyDescriptor.Builder() + .name("Max Pending Messages") + .description("Set the max size of the queue holding the messages pending to receive an " + + "acknowledgment from the broker.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1000") + .build(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + private LRUCache producers; + private ProducerConfiguration producerConfig; + + static { + final List properties = new ArrayList<>(); + properties.add(PULSAR_CLIENT_SERVICE); + properties.add(TOPIC); + properties.add(ASYNC_ENABLED); + properties.add(BATCHING_ENABLED); + properties.add(BATCHING_MAX_MESSAGES); + properties.add(BATCH_INTERVAL); + properties.add(BLOCK_IF_QUEUE_FULL); + properties.add(COMPRESSION_TYPE); + properties.add(MESSAGE_ROUTING_MODE); + properties.add(PENDING_MAX_MESSAGES); + + PROPERTIES = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + RELATIONSHIPS = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTIES; + } + + @OnStopped + public void cleanUp(final ProcessContext context) { + // Close all of the producers and invalidate them, so they get removed from the Resource Pool + getProducerCache(context).clear(); + } + + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + FlowFile flowFile = session.get(); + + if (flowFile == null) + return; + + final ComponentLog logger = getLogger(); + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + + if (StringUtils.isBlank(topic)) { + logger.error("Invalid topic specified {}", new Object[] {topic}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + // Read the contents of the FlowFile into a byte array + final byte[] messageContent = new byte[(int) flowFile.getSize()]; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + StreamUtils.fillBuffer(in, messageContent, true); + } + }); + + // Nothing to do, so skip this Flow file. + if (messageContent == null || messageContent.length < 1) { + session.transfer(flowFile, REL_SUCCESS); + return; + } + + try { + + Producer producer = getWrappedProducer(topic, context).getProducer(); + + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + this.sendAsync(producer, session, flowFile, messageContent); + } else { + this.send(producer, session, flowFile, messageContent); + } + + } catch (final PulsarClientException e) { + logger.error("Failed to connect to Pulsar Server due to {}", new Object[]{e}); + session.penalize(flowFile); + session.transfer(flowFile, REL_FAILURE); + } + + } + + + private void send(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) throws PulsarClientException { + + MessageId msgId = producer.send(messageContent); + + if (msgId != null) { + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } else { + session.transfer(flowFile, REL_FAILURE); + } + + } + + private void sendAsync(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) { + + producer.sendAsync(messageContent).handle((msgId, ex) -> { + if (msgId != null) { + return msgId; + } else { + // TODO Communicate the error back up to the onTrigger method so we can invalidate this producer. + getLogger().warn("Problem ", ex); + return null; + } + }); + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } + + private PulsarProducer getWrappedProducer(String topic, ProcessContext context) throws PulsarClientException, IllegalArgumentException { + + PulsarProducer producer = getProducerCache(context).get(topic); + + if (producer != null) + return producer; + + try { + producer = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool().acquire(getProducerProperties(context, topic)); + + if (producer != null) { + producers.put(topic, producer); + } + + return producer; + + } catch (InterruptedException e) { + return null; + } + + } + + private LRUCache getProducerCache(ProcessContext context) { + if (producers == null) { + + ResourcePool pool = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool(); + + producers = new LRUCache (20, pool); + } + + return producers; + } + + private Properties getProducerProperties(ProcessContext context, String topic) { + + Properties props = new Properties(); + props.put(PulsarProducerFactory.TOPIC_NAME, topic); + props.put(PulsarProducerFactory.PRODUCER_CONFIG, getProducerConfig(context)); + return props; + } + + private ProducerConfiguration getProducerConfig(ProcessContext context) { + + if (producerConfig == null) { + producerConfig = new ProducerConfiguration(); + + if (context.getProperty(BATCHING_ENABLED).isSet()) + producerConfig.setBatchingEnabled(context.getProperty(BATCHING_ENABLED).asBoolean()); + + if (context.getProperty(BATCHING_MAX_MESSAGES).isSet()) + producerConfig.setBatchingMaxMessages(context.getProperty(BATCHING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(BATCH_INTERVAL).isSet()) + producerConfig.setBatchingMaxPublishDelay(context.getProperty(BATCH_INTERVAL).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(BLOCK_IF_QUEUE_FULL).isSet()) + producerConfig.setBlockIfQueueFull(context.getProperty(BLOCK_IF_QUEUE_FULL).asBoolean()); + + if (context.getProperty(COMPRESSION_TYPE).isSet()) + producerConfig.setCompressionType(CompressionType.valueOf(context.getProperty(COMPRESSION_TYPE).getValue())); + + if (context.getProperty(PENDING_MAX_MESSAGES).isSet()) + producerConfig.setMaxPendingMessages(context.getProperty(PENDING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(MESSAGE_ROUTING_MODE).isSet()) + producerConfig.setMessageRoutingMode(MessageRoutingMode.valueOf(context.getProperty(MESSAGE_ROUTING_MODE).getValue())); + } + + return producerConfig; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index fbab5769250f..16a1d7132811 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -12,5 +12,10 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +<<<<<<< HEAD org.apache.nifi.processors.pulsar.ConsumePulsar_1_0 -org.apache.nifi.processors.pulsar.PublishPulsar_1_0 \ No newline at end of file +org.apache.nifi.processors.pulsar.PublishPulsar_1_0 +======= +org.apache.nifi.processors.pulsar.ConsumePulsar +org.apache.nifi.processors.pulsar.PublishPulsar +>>>>>>> Added Pulsar processors and Controller Service diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java index 1bee9dc120a5..2c61b4a3b396 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -32,6 +32,10 @@ protected void addPulsarClientService() throws InitializationException { final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); runner.addControllerService("pulsarClient", pulsarClient); runner.enableControllerService(pulsarClient); +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.PULSAR_CLIENT_SERVICE, "pulsarClient"); +======= + runner.setProperty(PublishPulsar.PULSAR_CLIENT_SERVICE, "pulsarClient"); +>>>>>>> Added Pulsar processors and Controller Service } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java index 84ae6a4a56e2..218b4b126ac9 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -49,7 +49,11 @@ public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { +<<<<<<< HEAD runner = TestRunners.newTestRunner(ConsumePulsar_1_0.class); +======= + runner = TestRunners.newTestRunner(ConsumePulsar.class); +>>>>>>> Added Pulsar processors and Controller Service mockClient = mock(PulsarClient.class); mockConsumer = mock(Consumer.class); @@ -77,10 +81,17 @@ public void init() throws InitializationException { public void emptyMessageTest() { when(mockMessage.getData()).thenReturn("".getBytes()); +<<<<<<< HEAD runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); runner.run(); runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); +======= + runner.setProperty(ConsumePulsar.TOPIC, "foo"); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.run(); + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); +>>>>>>> Added Pulsar processors and Controller Service } @Test @@ -110,10 +121,17 @@ public void multipleAsyncMessagesTest() throws PulsarClientException { public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); +<<<<<<< HEAD runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); runner.run(10, true); runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); +======= + runner.setProperty(ConsumePulsar.TOPIC, "foo"); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); + runner.run(10, true); + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); +>>>>>>> Added Pulsar processors and Controller Service runner.assertQueueEmpty(); @@ -132,6 +150,7 @@ private void sendMessages(String msg, String topic, String sub, boolean async, i when(mockMessage.getData()).thenReturn(msg.getBytes()); +<<<<<<< HEAD runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); @@ -140,6 +159,16 @@ private void sendMessages(String msg, String topic, String sub, boolean async, i runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_0.REL_SUCCESS); +======= + runner.setProperty(ConsumePulsar.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar.TOPIC, topic); + runner.setProperty(ConsumePulsar.SUBSCRIPTION, sub); + runner.run(itertions, true); + + runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar.REL_SUCCESS); +>>>>>>> Added Pulsar processors and Controller Service assertEquals(itertions, flowFiles.size()); for (MockFlowFile ff : flowFiles) { diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java index 0b15b0ae5ec8..754f47fe1d0e 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -44,7 +44,11 @@ public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { +<<<<<<< HEAD runner = TestRunners.newTestRunner(PublishPulsar_1_0.class); +======= + runner = TestRunners.newTestRunner(PublishPulsar.class); +>>>>>>> Added Pulsar processors and Controller Service mockClient = mock(PulsarClient.class); mockProducer = mock(Producer.class); @@ -82,6 +86,7 @@ public boolean matches(Object argument) { @Test public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); final String content = "some content"; @@ -91,6 +96,17 @@ public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClient runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); +======= + runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap (); + attributes.put(PublishPulsar.TOPIC.getName(), ""); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_FAILURE); +>>>>>>> Added Pulsar processors and Controller Service // Confirm that no Producer as created verify(mockClient, times(0)).createProducer(anyString()); @@ -99,6 +115,7 @@ public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); final String content = "some content"; @@ -108,6 +125,17 @@ public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClient runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +======= + runner.setProperty(PublishPulsar.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap (); + attributes.put(PublishPulsar.TOPIC.getName(), "topic-b"); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); +>>>>>>> Added Pulsar processors and Controller Service // Verify that we sent the data to topic-b. verify(mockClient, times(1)).createProducer("topic-b"); @@ -116,14 +144,24 @@ public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); +======= + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); +>>>>>>> Added Pulsar processors and Controller Service final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); +======= + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); +>>>>>>> Added Pulsar processors and Controller Service outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -136,15 +174,26 @@ public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClie @Test public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); +======= + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); +>>>>>>> Added Pulsar processors and Controller Service final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); +======= + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); +>>>>>>> Added Pulsar processors and Controller Service outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -157,14 +206,22 @@ public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, Pulsa @Test public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); +======= + runner.setProperty(PublishPulsar.TOPIC, "my-topic"); +>>>>>>> Added Pulsar processors and Controller Service final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +======= + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); +>>>>>>> Added Pulsar processors and Controller Service } // Verify that the send method on the producer was called with the expected content @@ -174,15 +231,24 @@ public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarC @Test public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); +======= + runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); +>>>>>>> Added Pulsar processors and Controller Service final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +======= + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); +>>>>>>> Added Pulsar processors and Controller Service } // Verify that the send method on the producer was called with the expected content @@ -191,14 +257,23 @@ public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, Pu @Test public void stressTest() throws UnsupportedEncodingException { +<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); +======= + runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); +>>>>>>> Added Pulsar processors and Controller Service final String content = "some content"; for (int idx = 0; idx < 9999; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +======= + runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); +>>>>>>> Added Pulsar processors and Controller Service } } diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore b/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore deleted file mode 100644 index 8cfc6f727089..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/.gitignore +++ /dev/null @@ -1,5 +0,0 @@ -.project -.settings/ -*/.classpath -*/.gitignore -/target/ diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml deleted file mode 100644 index 4e6ed6d98a03..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api-nar/pom.xml +++ /dev/null @@ -1,46 +0,0 @@ - - - - 4.0.0 - - - org.apache.nifi - nifi-pulsar-client-services - 1.6.0-SNAPSHOT - - - nifi-pulsar-client-service-api-nar - 1.6.0-SNAPSHOT - nar - - true - true - - - - - org.apache.nifi - nifi-standard-services-api-nar - nar - - - org.apache.nifi - nifi-pulsar-client-service-api - 1.6.0-SNAPSHOT - - - - diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml deleted file mode 100644 index 2259fc94f5ff..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/pom.xml +++ /dev/null @@ -1,40 +0,0 @@ - - - - 4.0.0 - - - org.apache.nifi - nifi-pulsar-client-services - 1.6.0-SNAPSHOT - - - nifi-pulsar-client-service-api - jar - - - - org.apache.nifi - nifi-api - provided - - - org.apache.pulsar - pulsar-client - 1.21.0-incubating - - - diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java deleted file mode 100644 index be619eae4297..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar; - -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.controller.ControllerService; -import org.apache.nifi.pulsar.pool.ResourcePool; - - -@Tags({"Pulsar"}) -@CapabilityDescription("Provides the ability to create Pulsar Producer / Consumer instances on demand, based on the configuration." - + "properties defined") -public interface PulsarClientPool extends ControllerService { - - public ResourcePool getProducerPool(); - - public ResourcePool getConsumerPool(); -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java deleted file mode 100644 index 59c98cdd0246..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar; - -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.pulsar.pool.PoolableResource; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.ConsumerStats; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@SuppressWarnings("unused") -public class PulsarConsumer implements PoolableResource { - - private static final Logger logger = LoggerFactory.getLogger(PulsarConsumer.class); - - private final Consumer consumer; - private final String topicName; - private final String subscriptionName; - private boolean closed = false; - - public PulsarConsumer(Consumer consumer, String topic, String subscription) throws PulsarClientException { - this.consumer = consumer; - this.topicName = topic; - this.subscriptionName = subscription; - } - - public void close() { - - logger.info("Closing consumer for topic {} and subscription {}", new Object[] {topicName, subscriptionName}); - closed = true; - - try { - consumer.unsubscribe(); - consumer.close(); - } catch (PulsarClientException e) { - logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); - closed = false; - } - } - - public boolean isClosed() { - return closed; - } - - public Consumer getConsumer() { - return this.consumer; - } - - public ConsumerStats getStats() { - return this.consumer.getStats(); - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java deleted file mode 100644 index a13975b270c2..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar; - -import org.apache.nifi.pulsar.pool.PoolableResource; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClientException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class PulsarProducer implements PoolableResource { - - private static final Logger logger = LoggerFactory.getLogger(PulsarProducer.class); - - private final Producer producer; - private boolean closed = false; - private final String topicName; - - public PulsarProducer(Producer producer, String topicName) throws PulsarClientException { - this.topicName = topicName; - this.producer = producer; - } - - public Producer getProducer() { - return producer; - } - - public String getName() { - return topicName; - } - - public boolean isClosed() { - return this.closed; - } - - public void close() { - - logger.info("Closing producer for topic {} ", new Object[] {topicName}); - - this.closed = true; - try { - producer.close(); - } catch (PulsarClientException e) { - logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); - closed = false; - } - - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java deleted file mode 100644 index 86a0fc2afab0..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.cache; - -import java.util.LinkedHashMap; - -import org.apache.nifi.pulsar.pool.PoolableResource; -import org.apache.nifi.pulsar.pool.ResourcePool; - -public class LRUCache { - - private LinkedHashMap lruCacheMap; - private final int capacity; - private final boolean SORT_BY_ACCESS = true; - private final float LOAD_FACTOR = 0.75F; - private final ResourcePool resourcePool; - - public LRUCache(int capacity, ResourcePool resourcePool){ - this.capacity = capacity; - this.lruCacheMap = new LinkedHashMap<>(capacity, LOAD_FACTOR, SORT_BY_ACCESS); - this.resourcePool = resourcePool; - } - - public V get(K k){ - return lruCacheMap.get(k); - } - - public void put(K k, V v){ - if(lruCacheMap.containsKey(k)){ - lruCacheMap.remove(k); - } else if(lruCacheMap.size() >= capacity){ - K victimKey = lruCacheMap.keySet().iterator().next(); - V victim = lruCacheMap.get(victimKey); - lruCacheMap.remove(victimKey); - resourcePool.evict(victim); - } - lruCacheMap.put(k, v); - } - - public int getSize() { - return lruCacheMap.size(); - } - - public void clear() { - for (V victim :lruCacheMap.values()) { - resourcePool.evict(victim); - } - - lruCacheMap.clear(); - } - - public void printSequence(){ - System.out.println(lruCacheMap.keySet()); - } -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java deleted file mode 100644 index 6af5845e0885..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -public interface PoolableResource { - - public void close(); - - public boolean isClosed(); - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java deleted file mode 100644 index 6794d2a7165e..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import java.util.Properties; - -import org.apache.nifi.pulsar.PulsarConsumer; -import org.apache.pulsar.client.api.ConsumerConfiguration; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; - -public class PulsarConsumerFactory implements ResourceFactory { - - public static final String TOPIC_NAME = "topic"; - public static final String SUBSCRIPTION_NAME = "subscription"; - public static final String CONSUMER_CONFIG = "Consumer-Configuration"; - - private PulsarClient client; - - public PulsarConsumerFactory(PulsarClient client) { - this.client = client; - } - - @Override - public PulsarConsumer create(Properties props) throws ResourceCreationException { - - String topic = props.getProperty(TOPIC_NAME); - String subscription = props.getProperty(SUBSCRIPTION_NAME); - ConsumerConfiguration config = (ConsumerConfiguration) props.get(CONSUMER_CONFIG); - - try { - // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice - return (config == null) ? new PulsarConsumer(client.subscribe(topic, subscription), topic, subscription) : - new PulsarConsumer(client.subscribe(topic, subscription, config), topic, subscription); - - } catch (PulsarClientException e) { - throw new ResourceCreationException(e); - } - - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java deleted file mode 100644 index 37165fc45ee6..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import java.util.Properties; - -import org.apache.nifi.pulsar.PulsarProducer; -import org.apache.pulsar.client.api.ProducerConfiguration; -import org.apache.pulsar.client.api.PulsarClient; - -public class PulsarProducerFactory implements ResourceFactory { - - public static final String TOPIC_NAME = "topic"; - public static final String PRODUCER_CONFIG = "Producer-Configuration"; - - private PulsarClient client; - - public PulsarProducerFactory(PulsarClient client) { - this.client = client; - } - - @Override - public PulsarProducer create(Properties props) throws ResourceCreationException { - - String topic = props.getProperty(TOPIC_NAME); - ProducerConfiguration config = (ProducerConfiguration) props.get(PRODUCER_CONFIG); - - try { - // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice - return (config == null) ? new PulsarProducer(client.createProducer(topic), topic) : - new PulsarProducer(client.createProducer(topic, config), topic); - - } catch (Exception e) { - throw new ResourceCreationException(e); - } - - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java deleted file mode 100644 index 7eb3ea6ccee5..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -public class ResourceCreationException extends Exception { - - private static final long serialVersionUID = -1796731800483756134L; - - public ResourceCreationException(String s) { - super(s); - } - - public ResourceCreationException(Exception e) { - super(e); - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java deleted file mode 100644 index 403d95cbcd71..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java +++ /dev/null @@ -1,23 +0,0 @@ - -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -public interface ResourceExceptionHandler { - - void handle(Exception exc); -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java deleted file mode 100644 index 1cf0500d441d..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import java.util.Properties; - -public interface ResourceFactory { - - public R create(Properties props) throws ResourceCreationException; -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java deleted file mode 100644 index e67b8f4b58e7..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import java.util.Properties; - -public interface ResourcePool { - - /** - * Acquire a resource from the pool. Creating one if necessary - * @param props - * @return - * @throws InterruptedException - */ - public R acquire(Properties props) throws InterruptedException; - - /** - * Evict the resource from the pool, destroying it. - * Call this method is the resource is known to be in an unusable state. - * @param resource - */ - public void evict(R resource); - - /** - * Place the resource back into the pool for future use. - * @param resource - */ - public void release(R resource); -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java deleted file mode 100644 index 53f3991b116d..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.cache; - -import java.util.Properties; - -import org.apache.nifi.pulsar.PulsarProducer; -import org.apache.nifi.pulsar.pool.ResourcePool; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.*; - -public class LRUCacheTest { - - @Mock - private ResourcePool mockedResourcePool; - - @Mock - private PulsarProducer mockedPulsarProducer; - - @SuppressWarnings("unchecked") - @Before - public void setUp() throws InterruptedException { - mockedResourcePool = mock(ResourcePool.class); - mockedPulsarProducer = mock(PulsarProducer.class); - - when(mockedResourcePool.acquire(any(Properties.class))).thenReturn(mockedPulsarProducer); - } - - /** - * Make sure the LRUCache functions as a Map - */ - @Test - public void simpleTest() { - LRUCache cache = new LRUCache (10, mockedResourcePool); - - for(Character i='A'; i<='E'; i++){ - cache.put(i.toString(), mockedPulsarProducer); - } - - assertEquals(5, cache.getSize()); - - for(Character i='A'; i<='E'; i++){ - assertNotNull( cache.get(i.toString())); - } - } - - @Test - public void evictionTest() { - - LRUCache cache = new LRUCache (5, mockedResourcePool); - - for(Character i='A'; i<='Z'; i++){ - cache.put(i.toString(), mockedPulsarProducer); - } - - // Make sure we only have 5 items in the cache - assertEquals(5, cache.getSize()); - - // Make sure we have the last 5 items added to the cache - for(Character i='V'; i<='Z'; i++){ - assertNotNull( cache.get(i.toString())); - } - - // Make sure the evict method on the resource pool was called 21 times, once for every item removed. - verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); - } - - @Test - public void evictionLruTest() { - - LRUCache cache = new LRUCache (5, mockedResourcePool); - - final Character A = 'A'; - - // Write 25 items to the cache, and the letter 'A' every other put. - for(Character i='B'; i<='Z'; i++){ - cache.put(i.toString(), mockedPulsarProducer); - cache.put(A.toString(), mockedPulsarProducer); - } - - // Make sure we only have 5 items in the cache - assertEquals(5, cache.getSize()); - - // Make sure that the letter 'A' is still in the cache due to frequent access - assertNotNull( cache.get(A.toString()) ); - - // Make sure we have the last 4 items added to the cache - for(Character i='W'; i<='Z'; i++){ - assertNotNull( cache.get(i.toString())); - } - - // Make sure the evict method on the resource pool was called 21 times, once for every item removed. - verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); - } - - @Test - public void clearTest() { - LRUCache cache = new LRUCache (26, mockedResourcePool); - - for(Character i='A'; i<='Z'; i++){ - cache.put(i.toString(), mockedPulsarProducer); - } - - // Make sure we only have all the items in the cache - assertEquals(26, cache.getSize()); - - cache.clear(); - - // Make sure all the items were removed - assertEquals(0, cache.getSize()); - - // Make sure all the items were evicted from the underlying resource pool - verify(mockedResourcePool, times(26)).evict(mockedPulsarProducer); - } -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml deleted file mode 100644 index 460e2e12febf..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service-nar/pom.xml +++ /dev/null @@ -1,47 +0,0 @@ - - - - 4.0.0 - - - org.apache.nifi - nifi-pulsar-client-services - 1.6.0-SNAPSHOT - - - nifi-pulsar-client-service-nar - 1.6.0-SNAPSHOT - nar - - true - true - - - - - org.apache.nifi - nifi-pulsar-client-service-api-nar - 1.6.0-SNAPSHOT - nar - - - org.apache.nifi - nifi-pulsar-client-service - 1.6.0-SNAPSHOT - - - - diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml deleted file mode 100644 index 5c3d3a6efc49..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/pom.xml +++ /dev/null @@ -1,64 +0,0 @@ - - - - 4.0.0 - - - org.apache.nifi - nifi-pulsar-client-services - 1.6.0-SNAPSHOT - - - nifi-pulsar-client-service - jar - - - - org.apache.nifi - nifi-pulsar-client-service-api - 1.6.0-SNAPSHOT - - - org.apache.nifi - nifi-api - provided - - - org.apache.nifi - nifi-processor-utils - - - org.apache.nifi - nifi-ssl-context-service-api - provided - - - org.apache.nifi - nifi-mock - test - - - org.slf4j - slf4j-simple - test - - - junit - junit - test - - - diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java deleted file mode 100644 index e965d99b89f0..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java +++ /dev/null @@ -1,300 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.controller.AbstractControllerService; -import org.apache.nifi.controller.ConfigurationContext; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; -import org.apache.nifi.pulsar.pool.PulsarProducerFactory; -import org.apache.nifi.pulsar.pool.ResourcePool; -import org.apache.nifi.pulsar.pool.ResourcePoolImpl; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.ssl.SSLContextService; -import org.apache.pulsar.client.api.ClientConfiguration; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException; -import org.apache.pulsar.client.impl.auth.AuthenticationTls; - -@Tags({ "Pulsar"}) -@CapabilityDescription("Standard ControllerService implementation of PulsarClientService.") -public class StandardPulsarClientPool extends AbstractControllerService implements PulsarClientPool { - - public static final PropertyDescriptor PULSAR_SERVICE_URL = new PropertyDescriptor - .Builder().name("PULSAR_SERVICE_URL") - .displayName("Pulsar Service URL") - .description("URL for the Pulsar cluster, e.g localhost:6650") - .required(true) - .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR) - .build(); - - public static final PropertyDescriptor CONCURRENT_LOOKUP_REQUESTS = new PropertyDescriptor.Builder() - .name("Maximum concurrent lookup-requests") - .description("Number of concurrent lookup-requests allowed on each broker-connection to prevent " - + "overload on broker. (default: 5000) It should be configured with higher value only in case " - + "of it requires to produce/subscribe on thousands of topics") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("5000") - .build(); - - public static final PropertyDescriptor CONNECTIONS_PER_BROKER = new PropertyDescriptor.Builder() - .name("Maximum connects per Pulsar broker") - .description("Sets the max number of connection that the client library will open to a single broker.\n" + - "By default, the connection pool will use a single connection for all the producers and consumers. " - + "Increasing this parameter may improve throughput when using many producers over a high latency connection") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("1") - .build(); - - public static final PropertyDescriptor IO_THREADS = new PropertyDescriptor.Builder() - .name("I/O Threads") - .description("The number of threads to be used for handling connections to brokers (default: 1 thread)") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("1") - .build(); - - public static final PropertyDescriptor LISTENER_THREADS = new PropertyDescriptor.Builder() - .name("Listener Threads") - .description("The number of threads to be used for message listeners (default: 1 thread)") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("1") - .build(); - - public static final PropertyDescriptor MAXIMUM_REJECTED_REQUESTS = new PropertyDescriptor.Builder() - .name("Maximum rejected requests per connection") - .description("Max number of broker-rejected requests in a certain time-frame (30 seconds) after " - + "which current connection will be closed and client creates a new connection that give " - + "chance to connect a different broker (default: 50)") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("50") - .build(); - - public static final PropertyDescriptor OPERATION_TIMEOUT = new PropertyDescriptor.Builder() - .name("Operation Timeout") - .description("Producer-create, subscribe and unsubscribe operations will be retried until this " - + "interval, after which the operation will be maked as failed (default: 30 seconds)") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("30") - .build(); - - public static final PropertyDescriptor STATS_INTERVAL = new PropertyDescriptor.Builder() - .name("Stats interval") - .description("The interval between each stat info (default: 60 seconds) Stats will be activated " - + "with positive statsIntervalSeconds It should be set to at least 1 second") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("60") - .build(); - - public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder() - .name("Use TCP nodelay flag") - .description("Configure whether to use TCP no-delay flag on the connection, to disable Nagle algorithm.\n" - + "No-delay features make sure packets are sent out on the network as soon as possible, and it's critical " - + "to achieve low latency publishes. On the other hand, sending out a huge number of small packets might " - + "limit the overall throughput, so if latency is not a concern, it's advisable to set the useTcpNoDelay " - + "flag to false.") - .required(false) - .addValidator(StandardValidators.BOOLEAN_VALIDATOR) - .defaultValue("false") - .build(); - - public static final PropertyDescriptor MAX_PRODUCERS = new PropertyDescriptor - .Builder().name("MAX_PRODUCERS") - .displayName("Producer Pool Size") - .description("The Maximum Number of Pulsar Producers created by this Pulsar Client Pool") - .required(true) - .defaultValue("10") - .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) - .build(); - - public static final PropertyDescriptor MAX_CONSUMERS = new PropertyDescriptor - .Builder().name("MAX_CONSUMERS") - .displayName("Consumer Pool Size") - .description("The Maximum Number of Pulsar consumers created by this Pulsar Client Pool") - .required(true) - .defaultValue("10") - .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) - .build(); - - public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() - .name("ssl.context.service") - .displayName("SSL Context Service") - .description("Specifies the SSL Context Service to use for communicating with Pulsar.") - .required(false) - .identifiesControllerService(SSLContextService.class) - .build(); - - private static final List properties; - private volatile PulsarClient client; - - private volatile ResourcePoolImpl producers; - private volatile ResourcePoolImpl consumers; - private ClientConfiguration clientConfig; - - static { - final List props = new ArrayList<>(); - props.add(PULSAR_SERVICE_URL); - props.add(MAX_CONSUMERS); - props.add(MAX_PRODUCERS); - props.add(CONCURRENT_LOOKUP_REQUESTS); - props.add(CONNECTIONS_PER_BROKER); - props.add(IO_THREADS); - props.add(LISTENER_THREADS); - props.add(MAXIMUM_REJECTED_REQUESTS); - props.add(OPERATION_TIMEOUT); - props.add(STATS_INTERVAL); - props.add(USE_TCP_NO_DELAY); - properties = Collections.unmodifiableList(props); - } - - @Override - protected List getSupportedPropertyDescriptors() { - return properties; - } - - /** - * @param context - * the configuration context - * @throws InitializationException - * if unable to create a database connection - */ - @OnEnabled - public void onEnabled(final ConfigurationContext context) throws InitializationException { - - createClient(context); - - if (this.client == null) - throw new InitializationException("Unable to create Pulsar Client"); - - producers = new ResourcePoolImpl(new PulsarProducerFactory(client), context.getProperty(MAX_PRODUCERS).asInteger()); - consumers = new ResourcePoolImpl(new PulsarConsumerFactory(client), context.getProperty(MAX_CONSUMERS).asInteger()); - - } - - private void createClient(final ConfigurationContext context) throws InitializationException { - - // We can't create a client without a service URL. - if (!context.getProperty(PULSAR_SERVICE_URL).isSet()) { - return; - } - - try { - this.client = PulsarClient.create(buildPulsarBrokerRootUrl(context.getProperty(PULSAR_SERVICE_URL).getValue(), - getClientConfig(context).isUseTls()), getClientConfig(context)); - - } catch (Exception e) { - throw new InitializationException("Unable to create Pulsar Client", e); - } - - } - - private static String buildPulsarBrokerRootUrl(String uri, boolean tlsEnabled) { - StringBuilder builder = new StringBuilder(); - builder.append("pulsar"); - - if (tlsEnabled) - builder.append("+ssl"); - - builder.append("://"); - builder.append(uri); - return builder.toString(); - } - - private ClientConfiguration getClientConfig(ConfigurationContext context) throws UnsupportedAuthenticationException { - - if (clientConfig == null) { - clientConfig = new ClientConfiguration(); - - if (context.getProperty(CONCURRENT_LOOKUP_REQUESTS).isSet()) { - clientConfig.setConcurrentLookupRequest(context.getProperty(CONCURRENT_LOOKUP_REQUESTS).asInteger()); - } - - if (context.getProperty(CONNECTIONS_PER_BROKER).isSet()) { - clientConfig.setConnectionsPerBroker(context.getProperty(CONNECTIONS_PER_BROKER).asInteger()); - } - - if (context.getProperty(IO_THREADS).isSet()) { - clientConfig.setIoThreads(context.getProperty(IO_THREADS).asInteger()); - } - - if (context.getProperty(LISTENER_THREADS).isSet()) { - clientConfig.setListenerThreads(context.getProperty(LISTENER_THREADS).asInteger()); - } - - if (context.getProperty(MAXIMUM_REJECTED_REQUESTS).isSet()) { - clientConfig.setMaxNumberOfRejectedRequestPerConnection(context.getProperty(MAXIMUM_REJECTED_REQUESTS).asInteger()); - } - - if (context.getProperty(OPERATION_TIMEOUT).isSet()) { - clientConfig.setOperationTimeout(context.getProperty(OPERATION_TIMEOUT).asInteger(), TimeUnit.SECONDS); - } - - if (context.getProperty(STATS_INTERVAL).isSet()) { - clientConfig.setStatsInterval(context.getProperty(STATS_INTERVAL).asLong(), TimeUnit.SECONDS); - } - - if (context.getProperty(USE_TCP_NO_DELAY).isSet()) { - clientConfig.setUseTcpNoDelay(context.getProperty(USE_TCP_NO_DELAY).asBoolean()); - } - - // Configure TLS - final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); - - if (sslContextService != null && sslContextService.isTrustStoreConfigured() && sslContextService.isKeyStoreConfigured()) { - clientConfig.setUseTls(true); - clientConfig.setTlsTrustCertsFilePath(sslContextService.getTrustStoreFile()); - - Map authParams = new HashMap<>(); - - // TODO This should be a different value than the TlsTrustCertsFilePath above. - authParams.put("tlsCertFile", sslContextService.getTrustStoreFile()); - authParams.put("tlsKeyFile", sslContextService.getKeyStoreFile()); - clientConfig.setAuthentication(AuthenticationTls.class.getName(), authParams); - } - } - - return clientConfig; - } - - @Override - public ResourcePool getProducerPool() { - return this.producers; - } - - @Override - public ResourcePool getConsumerPool() { - return this.consumers; - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java deleted file mode 100644 index 86d4934278d2..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ResourceExceptionHandlerImpl implements ResourceExceptionHandler { - - private static Logger logger = LoggerFactory.getLogger(ResourceExceptionHandlerImpl.class); - - @Override - public void handle(Exception exc) { - logger.error("Unable to create Resource", exc); - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java deleted file mode 100644 index 90ddb95aca13..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import java.util.Iterator; -import java.util.Properties; -import java.util.Vector; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - - -public class ResourcePoolImpl implements ResourcePool { - - private final Lock lock = new ReentrantLock(); - private final Condition poolAvailable = lock.newCondition(); - private int max_resources; - private final Vector pool; - - private final ResourceExceptionHandler resourceExceptionHandler; - private final ResourceFactory factory; - - public ResourcePoolImpl(ResourceFactory factory, int max_resources) { - this(factory, new ResourceExceptionHandlerImpl(), max_resources); - } - - public ResourcePoolImpl(ResourceFactory factory, ResourceExceptionHandler handler, int max_resources) { - lock.lock(); - try { - this.factory = factory; - this.resourceExceptionHandler = handler; - this.max_resources = max_resources; - this.pool = new Vector (max_resources); - } finally { - lock.unlock(); - } - } - - private R createResource(Properties props) { - R resource = null; - try { - - resource = factory.create(props); - - if (resource == null) - throw new ResourceCreationException("Unable to create resource"); - - } catch (Exception e) { - resourceExceptionHandler.handle(e); - } - return resource; - } - - - /* - * Shutdown the pool and release the resources - */ - public void close() { - - Iterator itr = pool.iterator(); - while (itr.hasNext()) { - itr.next().close(); - } - - } - - public boolean isEmpty() { - return (pool.isEmpty()); - } - - public boolean isFull() { - return (pool != null && pool.size() == max_resources); - } - - @Override - public R acquire(Properties props) throws InterruptedException { - lock.lock(); - try { - while (max_resources <= 0) { - poolAvailable.await(); - } - - - --max_resources; - - if (pool != null) { - int size = pool.size(); - if (size > 0) - return pool.remove(size - 1); - } - return createResource(props); - } finally { - lock.unlock(); - } - } - - @Override - public void evict(R resource) { - lock.lock(); - try { - - // Attempt to close the connection - if (!resource.isClosed()) - resource.close(); - - pool.removeElement(resource); - --max_resources; - poolAvailable.signal(); - } finally { - lock.unlock(); - } - } - - @Override - public void release(R resource) { - lock.lock(); - try { - pool.addElement(resource); - ++max_resources; - poolAvailable.signal(); - } finally { - lock.unlock(); - } - } -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService deleted file mode 100644 index c1eecc70a562..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService +++ /dev/null @@ -1,15 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -org.apache.nifi.pulsar.StandardPulsarClientPool \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java deleted file mode 100644 index 72d17006e83f..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.nifi.components.PropertyDescriptor; -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; - -public class TestProcessor extends AbstractProcessor { - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - } - - @Override - protected List getSupportedPropertyDescriptors() { - List propDescs = new ArrayList<>(); - propDescs.add(new PropertyDescriptor.Builder() - .name("StandardPulsarClientService test processor") - .description("StandardPulsarClientService test processor") - .identifiesControllerService(StandardPulsarClientPool.class) - .required(true) - .build()); - return propDescs; - } -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java deleted file mode 100644 index 4b9982df30db..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar; - -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.util.TestRunner; -import org.apache.nifi.util.TestRunners; -import org.junit.Before; -import org.junit.Test; - -public class TestStandardPulsarClientService { - - @Before - public void init() { - - } - - @Test - public void testService() throws InitializationException { - final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class); - final PulsarClientPool service = new StandardPulsarClientPool(); - runner.addControllerService("test-good", service); - - runner.setProperty(service, StandardPulsarClientPool.PULSAR_SERVICE_URL, "localhost:6667"); - // runner.enableControllerService(service); - - runner.assertValid(service); - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java deleted file mode 100644 index a19fbad556d3..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import org.apache.nifi.controller.AbstractControllerService; -import org.apache.nifi.pulsar.PulsarClientPool; -import org.apache.nifi.pulsar.PulsarConsumer; -import org.apache.nifi.pulsar.PulsarProducer; -import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; -import org.apache.nifi.pulsar.pool.PulsarProducerFactory; -import org.apache.nifi.pulsar.pool.ResourcePool; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.junit.Rule; -import org.mockito.ArgumentMatcher; -import org.mockito.Matchers; -import org.mockito.Mock; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.junit.MockitoJUnit; -import org.mockito.junit.MockitoRule; -import org.mockito.stubbing.Answer; - -import static org.mockito.Mockito.*; - -import java.util.Properties; - -public class MockPulsarClientService extends AbstractControllerService implements PulsarClientPool { - - @Mock - PulsarClient mockClient; - - @Mock - ResourcePool mockProducerPool; - - @Mock - ResourcePool mockConsumerPool; - - @Mock - Producer mockProducer; - - @Mock - Consumer mockConsumer; - - @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); - - @SuppressWarnings("unchecked") - public MockPulsarClientService() { - this.mockClient = mock(PulsarClient.class); - mockProducerPool = mock(ResourcePool.class); - mockConsumerPool = mock(ResourcePool.class); - mockProducer = mock(Producer.class); - mockConsumer = mock(Consumer.class); - - try { - when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( - new Answer () { - @Override - public PulsarProducer answer(InvocationOnMock invocation) { - Properties props = invocation.getArgumentAt(0, Properties.class); - return getProducer(props); - } - } - ); - - when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( - new Answer () { - @Override - public PulsarConsumer answer(InvocationOnMock invocation) { - Properties props = invocation.getArgumentAt(0, Properties.class); - return getConsumer(props); - } - } - ); - } catch (InterruptedException ex) { - - } - - try { - when(mockClient.createProducer(anyString())).thenReturn(getMockProducer()); - when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { - @Override - public boolean matches(Object argument) { - return true; - } - }))).thenReturn(null); - } catch (PulsarClientException e) { - e.printStackTrace(); - } - } - - - public Producer getMockProducer() { - return mockProducer; - } - - public PulsarClient getMockClient() { - return mockClient; - } - - public PulsarProducer getProducer(Properties props) { - String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); - try { - return new PulsarProducer(mockClient.createProducer(topic), topic); - } catch (PulsarClientException e) { - return null; - } - } - - public PulsarConsumer getConsumer(Properties props) { - String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); - String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); - try { - return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); - } catch (PulsarClientException e) { - return null; - } - } - - @Override - public ResourcePool getProducerPool() { - return mockProducerPool; - } - - - @Override - public ResourcePool getConsumerPool() { - return mockConsumerPool; - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java b/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java deleted file mode 100644 index 397b53880351..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.pulsar.pool; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.util.Properties; - -import org.apache.nifi.pulsar.PulsarProducer; -import org.junit.Test; - -public class ResourcePoolTest { - - private MockPulsarClientService pulsarClient = new MockPulsarClientService(); - - /*Checks the number of the resources in the Resource Table. - Resource is created only if needed.*/ - @Test - public void testIsEmptyBeforeResourceAcquired() throws Exception { - - ResourcePoolImpl resPool = getResourcePool("topic-a", 5); - - assertTrue(resPool.isEmpty()); - } - - /* Checks if the number of resources in the Resource Table is equal to the - * maximum number of resources declared at the Resource Pool creation time.*/ - @Test - public void testIsFull() throws Exception { - - Properties props = new Properties(); - props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); - - ResourcePoolImpl resPool = getResourcePool("topic-a", 5); - - PulsarProducer[] res = new PulsarProducer[5]; - for (int i = 0; i <= 4; i++) { - res[i] = resPool.acquire(props); - } - - assertTrue(resPool.isFull()); - - } - - /* - * Checks to see if resources are left in the pool after they - * have been acquired, and then released. - */ - @Test - public void testResourcesReused() throws InterruptedException { - Properties props = new Properties(); - props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); - - ResourcePoolImpl resPool = getResourcePool("topic-a", 5); - - PulsarProducer[] res = new PulsarProducer[5]; - for (int i = 0; i <= 4; i++) { - res[i] = resPool.acquire(props); - } - - assertTrue(resPool.isFull()); - - for (int i = 0; i <= 4; i++) { - resPool.release(res[i]); - } - - assertFalse(resPool.isEmpty()); - } - - /* - * Checks to see if resources that are invalidated are removed from - * the pool. - */ - @Test - public void testResourcesReleased() throws InterruptedException { - Properties props = new Properties(); - props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); - - ResourcePoolImpl resPool = getResourcePool("topic-a", 5); - - PulsarProducer[] res = new PulsarProducer[5]; - for (int i = 0; i <= 4; i++) { - res[i] = resPool.acquire(props); - } - - assertTrue(resPool.isFull()); - - for (int i = 0; i <= 4; i++) { - resPool.evict(res[i]); - } - - assertTrue(resPool.isEmpty()); - } - - @Test - public void testAcquireBlocksWhenEmpty() throws Exception { - - Properties props = new Properties(); - props.setProperty(PulsarProducerFactory.TOPIC_NAME, "A"); - - final ResourcePoolImpl resPool = new ResourcePoolImpl( - new PulsarProducerFactory(pulsarClient.getMockClient()) , 0); - - Thread resourceConsumer = new Thread() { - public void run() { - try { - @SuppressWarnings("unused") - PulsarProducer unused = resPool.acquire(props); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - fail(); // error if control flow reaches this line - } - }; - - resourceConsumer.start(); - Thread.sleep(1000); // waits for the resourceConsumer to block - resourceConsumer.interrupt(); - resourceConsumer.join(1000); // resume after the resourceConsumer ends - assertFalse(resourceConsumer.isAlive()); - - } - - - private ResourcePoolImpl getResourcePool(String topic, int size) { - return new ResourcePoolImpl( - new PulsarProducerFactory(pulsarClient.getMockClient()), size); - } -} diff --git a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml b/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml deleted file mode 100644 index 37b8e3fa3e54..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-client-services/pom.xml +++ /dev/null @@ -1,37 +0,0 @@ - - - - 4.0.0 - - - org.apache.nifi - nifi-nar-bundles - 1.6.0-SNAPSHOT - - - org.apache.nifi - nifi-pulsar-client-services - 1.6.0-SNAPSHOT - pom - - - nifi-pulsar-client-service-api - nifi-pulsar-client-service-api-nar - nifi-pulsar-client-service - nifi-pulsar-client-service-nar - - - From 475491195d2f1fc2c24a4f403d6fc6cfd30982b0 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Mon, 26 Feb 2018 11:19:26 -0800 Subject: [PATCH 08/16] Updated all nifi component version references to 1.6.0-SNAPSHOT --- nifi-nar-bundles/nifi-solr-bundle/nifi-solr-nar/pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-nar/pom.xml b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-nar/pom.xml index aac31b941164..40dbe8408690 100644 --- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-nar/pom.xml +++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-nar/pom.xml @@ -23,6 +23,8 @@ nifi-solr-nar + 1.6.0-SNAPSHOT + nar true @@ -39,7 +41,7 @@ org.apache.nifi nifi-standard-services-api-nar nar - + From 046e28703914f66efbea27a6112bc88b33750bdf Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Wed, 28 Feb 2018 10:18:35 -0800 Subject: [PATCH 09/16] Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 --- .../nifi/processors/pulsar/ConsumePulsar.java | 391 ------------------ .../nifi/processors/pulsar/PublishPulsar.java | 372 ----------------- .../org.apache.nifi.processor.Processor | 6 +- .../pulsar/AbstractPulsarProcessorTest.java | 4 - .../pulsar/ConsumePulsarProcessorTest.java | 32 +- .../pulsar/PublishPulsarProcessorTest.java | 74 +++- 6 files changed, 94 insertions(+), 785 deletions(-) delete mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java delete mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java deleted file mode 100644 index b84cb7aa70c5..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar.java +++ /dev/null @@ -1,391 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.pulsar; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - -import org.apache.nifi.annotation.behavior.InputRequirement; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnScheduled; -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.annotation.lifecycle.OnUnscheduled; -import org.apache.nifi.components.AllowableValue; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.pulsar.PulsarClientPool; -import org.apache.nifi.pulsar.PulsarConsumer; -import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerConfiguration; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.SubscriptionType; - -@Tags({"Pulsar", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume"}) -@CapabilityDescription("Consumes messages from Apache Pulsar " - + "The complementary NiFi processor for sending messages is PublishPulsar.") -@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) -public class ConsumePulsar extends AbstractPulsarProcessor { - - static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); - static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); - static final AllowableValue FAILOVER = new AllowableValue("Failover", "Failover", "Multiple consumer will be able to use the same subscription name but only 1 consumer " - + "will receive the messages. If that consumer disconnects, one of the other connected consumers will start receiving messages"); - - protected static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() - .name("topic") - .displayName("Topic Name") - .description("The name of the Pulsar Topic.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - static final PropertyDescriptor SUBSCRIPTION = new PropertyDescriptor.Builder() - .name("Subscription") - .displayName("Subscription Name") - .description("The name of the Pulsar subscription to consume from.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() - .name("Async Enabled") - .description("Control whether the messages will be consumed asyncronously or not. Messages consumed" - + " syncronously will be acknowledged immediately before processing the next message, while" - + " asyncronous messages will be acknowledged after the Pulsar broker responds.") - .required(true) - .addValidator(StandardValidators.BOOLEAN_VALIDATOR) - .defaultValue("false") - .build(); - - public static final PropertyDescriptor MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder() - .name("Maximum Async Requests") - .description("The maximum number of outstanding asynchronous consumer requests for this processor. " - + "Each asynchronous call requires memory, so avoid setting this value to high.") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("50") - .build(); - - public static final PropertyDescriptor ACK_TIMEOUT = new PropertyDescriptor.Builder() - .name("Acknowledgment Timeout") - .description("Set the timeout (in milliseconds) for unacked messages, truncated to the " - + "nearest millisecond. The timeout needs to be greater than 10 seconds.") - .required(false) - .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) - .defaultValue("10000") - .build(); - - public static final PropertyDescriptor PRIORITY_LEVEL = new PropertyDescriptor.Builder() - .name("Consumer Priority Level") - .description("Sets priority level for the shared subscription consumers to which broker " - + "gives more priority while dispatching messages. Here, broker follows descending " - + "priorities. (eg: 0=max-priority, 1, 2,..) ") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("5") - .build(); - - public static final PropertyDescriptor RECEIVER_QUEUE_SIZE = new PropertyDescriptor.Builder() - .name("Consumer receive queue size.") - .description("The consumer receive queue controls how many messages can be accumulated " - + "by the Consumer before the application calls Consumer.receive(). Using a higher " - + "value could potentially increase the consumer throughput at the expense of bigger " - + "memory utilization. \n" - + "Setting the consumer queue size as zero, \n" - + "\t - Decreases the throughput of the consumer, by disabling pre-fetching of messages. \n" - + "\t - Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer " - + "connection with broker and consumer will not be able receive any further message unless batch-message " - + "in pipeline is removed") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("1000") - .build(); - - public static final PropertyDescriptor SUBSCRIPTION_TYPE = new PropertyDescriptor.Builder() - .name("Subscription Type") - .description("Select the subscription type to be used when subscribing to the topic.") - .required(false) - .allowableValues(EXCLUSIVE, SHARED, FAILOVER) - .defaultValue(SHARED.getValue()) - .build(); - - private static final List PROPERTIES; - private static final Set RELATIONSHIPS; - - // Reuse the same consumer for a given topic / subscription - private PulsarConsumer consumer; - private ConsumerConfiguration consumerConfig; - - // Pool for running multiple consume Async requests - ExecutorService pool; - ExecutorCompletionService completionService; - - static { - final List properties = new ArrayList<>(); - properties.add(PULSAR_CLIENT_SERVICE); - properties.add(TOPIC); - properties.add(SUBSCRIPTION); - properties.add(ASYNC_ENABLED); - properties.add(MAX_ASYNC_REQUESTS); - properties.add(ACK_TIMEOUT); - properties.add(PRIORITY_LEVEL); - properties.add(RECEIVER_QUEUE_SIZE); - properties.add(SUBSCRIPTION_TYPE); - - PROPERTIES = Collections.unmodifiableList(properties); - - final Set relationships = new HashSet<>(); - relationships.add(REL_SUCCESS); - RELATIONSHIPS = Collections.unmodifiableSet(relationships); - } - - @Override - public Set getRelationships() { - return RELATIONSHIPS; - } - - @Override - protected List getSupportedPropertyDescriptors() { - return PROPERTIES; - } - - @OnScheduled - public void init(ProcessContext context) { - pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); - completionService = new ExecutorCompletionService<>(pool); - } - - @OnUnscheduled - public void shutDown() { - // Stop all the async consumers - pool.shutdownNow(); - } - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - - try { - if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { - // Launch consumers - consumeAsync(context, session); - - // Handle completed consumers - handleAsync(context, session); - - } else { - consume(context, session); - } - } catch (PulsarClientException e) { - getLogger().error("Unable to consume from Pulsar Topic ", e); - context.yield(); - throw new ProcessException(e); - } - - } - - private void handleAsync(ProcessContext context, ProcessSession session) { - - try { - Future done = completionService.take(); - Message msg = done.get(); - - if (msg != null) { - FlowFile flowFile = null; - final byte[] value = msg.getData(); - if (value != null && value.length > 0) { - flowFile = session.create(); - flowFile = session.write(flowFile, out -> { - out.write(value); - }); - } - - session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); - session.transfer(flowFile, REL_SUCCESS); - session.commit(); - getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); - } - - } catch (InterruptedException | ExecutionException | PulsarClientException e) { - getLogger().error("Trouble consuming messages ", e); - } - - } - - @OnStopped - public void close(final ProcessContext context) { - - getLogger().info("Disconnecting Pulsar Consumer"); - if (consumer != null) { - - context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class) - .getConsumerPool().evict(consumer); - } - - consumer = null; - } - - /* - * For now let's assume that this processor will be configured to run for a longer - * duration than 0 milliseconds. So we will be grabbing as many messages off the topic - * as possible and committing them as FlowFiles - */ - private void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException { - - Consumer consumer = getWrappedConsumer(context).getConsumer(); - - completionService.submit(new Callable() { - @Override - public Message call() throws Exception { - return consumer.receiveAsync().get(); - } - }); - - } - - /* - * When this Processor expects to receive many small files, it may - * be advisable to create several FlowFiles from a single session - * before committing the session. Typically, this allows the Framework - * to treat the content of the newly created FlowFiles much more efficiently. - */ - private void consume(ProcessContext context, ProcessSession session) throws PulsarClientException { - - Consumer consumer = getWrappedConsumer(context).getConsumer(); - - final ComponentLog logger = getLogger(); - final Message msg; - FlowFile flowFile = null; - - try { - - msg = consumer.receive(); - final byte[] value = msg.getData(); - - if (value != null && value.length > 0) { - flowFile = session.create(); - flowFile = session.write(flowFile, out -> { - out.write(value); - }); - - session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); - session.transfer(flowFile, REL_SUCCESS); - logger.info("Created {} from {} messages received from Pulsar Server and transferred to 'success'", - new Object[]{flowFile, 1}); - - session.commit(); - - /* - * This Processor acknowledges receipt of the data and/or removes the data - * from the external source in order to prevent receipt of duplicate files. - * This is done only after the ProcessSession by which the FlowFile was created - * has been committed! Failure to adhere to this principle may result in data - * loss, as restarting NiFi before the session has been committed will result - * in the temporary file being deleted. Note, however, that it is possible using - * this approach to receive duplicate data because the application could be - * restarted after committing the session and before acknowledging or removing - * the data from the external source. In general, though, potential data duplication - * is preferred over potential data loss. - */ - getLogger().info("Acknowledging message " + msg.getMessageId()); - consumer.acknowledge(msg); - - } else { - // We didn't consume any data, so - session.commit(); - } - - } catch (PulsarClientException e) { - context.yield(); - session.rollback(); - } - - } - - private PulsarConsumer getWrappedConsumer(ProcessContext context) throws PulsarClientException { - - if (consumer != null) - return consumer; - - final PulsarClientPool pulsarClientService = context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class); - - try { - consumer = pulsarClientService.getConsumerPool() - .acquire(getConsumerProperties(context)); - - if (consumer == null || consumer.getConsumer() == null) { - throw new PulsarClientException("Unable to create Pulsar Consumer"); - } - - return consumer; - } catch (final InterruptedException ex) { - return null; - } - } - - private Properties getConsumerProperties(ProcessContext context) { - - Properties props = new Properties(); - props.put(PulsarConsumerFactory.TOPIC_NAME, context.getProperty(TOPIC).getValue()); - props.put(PulsarConsumerFactory.SUBSCRIPTION_NAME, context.getProperty(SUBSCRIPTION).getValue()); - props.put(PulsarConsumerFactory.CONSUMER_CONFIG, getConsumerConfig(context)); - return props; - } - - private ConsumerConfiguration getConsumerConfig(ProcessContext context) { - - if (consumerConfig == null) { - consumerConfig = new ConsumerConfiguration(); - - if (context.getProperty(ACK_TIMEOUT).isSet()) - consumerConfig.setAckTimeout(context.getProperty(ACK_TIMEOUT).asLong(), TimeUnit.MILLISECONDS); - - if (context.getProperty(PRIORITY_LEVEL).isSet()) - consumerConfig.setPriorityLevel(context.getProperty(PRIORITY_LEVEL).asInteger()); - - if (context.getProperty(RECEIVER_QUEUE_SIZE).isSet()) - consumerConfig.setReceiverQueueSize(context.getProperty(RECEIVER_QUEUE_SIZE).asInteger()); - - if (context.getProperty(SUBSCRIPTION_TYPE).isSet()) - consumerConfig.setSubscriptionType(SubscriptionType.valueOf(context.getProperty(SUBSCRIPTION_TYPE).getValue())); - } - - return consumerConfig; - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java deleted file mode 100644 index 20b93ddfd234..000000000000 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar.java +++ /dev/null @@ -1,372 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.pulsar; - -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import org.apache.nifi.annotation.behavior.InputRequirement; -import org.apache.nifi.annotation.behavior.WritesAttribute; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.components.AllowableValue; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.pulsar.PulsarClientPool; -import org.apache.nifi.pulsar.PulsarProducer; -import org.apache.nifi.pulsar.cache.LRUCache; -import org.apache.nifi.pulsar.pool.PulsarProducerFactory; -import org.apache.nifi.pulsar.pool.ResourcePool; -import org.apache.nifi.stream.io.StreamUtils; -import org.apache.nifi.util.StringUtils; -import org.apache.pulsar.client.api.CompressionType; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerConfiguration; -import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode; -import org.apache.pulsar.client.api.PulsarClientException; - -@Tags({"Apache", "Pulsar", "Put", "Send", "Message", "PubSub"}) -@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Pulsar using the Pulsar 1.21 Producer API." - + "The messages to send may be individual FlowFiles or may be delimited, using a " - + "user-specified delimiter, such as a new-line. " - + "The complementary NiFi processor for fetching messages is ConsumePulsar.") -@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) -@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " - + "FlowFiles that are routed to success.") -public class PublishPulsar extends AbstractPulsarProcessor { - - protected static final String MSG_COUNT = "msg.count"; - - static final AllowableValue COMPRESSION_TYPE_NONE = new AllowableValue("NONE", "None", "No compression"); - static final AllowableValue COMPRESSION_TYPE_LZ4 = new AllowableValue("LZ4", "LZ4", "Compress with LZ4 algorithm."); - static final AllowableValue COMPRESSION_TYPE_ZLIB = new AllowableValue("ZLIB", "ZLIB", "Compress with ZLib algorithm"); - - static final AllowableValue MESSAGE_ROUTING_MODE_CUSTOM_PARTITION = new AllowableValue("CustomPartition", "Custom Partition", "Route messages to a custom partition"); - static final AllowableValue MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION = new AllowableValue("RoundRobinPartition", "Round Robin Partition", "Route messages to all partitions in a round robin manner"); - static final AllowableValue MESSAGE_ROUTING_MODE_SINGLE_PARTITION = new AllowableValue("SinglePartition", "Single Partition", "Route messages to a single partition"); - - public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() - .name("topic") - .displayName("Topic Name") - .description("The name of the Pulsar Topic.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .expressionLanguageSupported(true) - .build(); - - public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() - .name("Async Enabled") - .description("Control whether the messages will be sent asyncronously or not. Messages sent" - + " syncronously will be acknowledged immediately before processing the next message, while" - + " asyncronous messages will be acknowledged after the Pulsar broker responds.") - .required(true) - .addValidator(StandardValidators.BOOLEAN_VALIDATOR) - .defaultValue("false") - .build(); - - public static final PropertyDescriptor BATCHING_ENABLED = new PropertyDescriptor.Builder() - .name("Batching Enabled") - .description("Control whether automatic batching of messages is enabled for the producer. " - + "default: false [No batching] When batching is enabled, multiple calls to " - + "Producer.sendAsync can result in a single batch to be sent to the broker, leading " - + "to better throughput, especially when publishing small messages. If compression is " - + "enabled, messages will be compressed at the batch level, leading to a much better " - + "compression ratio for similar headers or contents. When enabled default batch delay " - + "is set to 10 ms and default batch size is 1000 messages") - .required(false) - .addValidator(StandardValidators.BOOLEAN_VALIDATOR) - .defaultValue("false") - .build(); - - public static final PropertyDescriptor BATCHING_MAX_MESSAGES = new PropertyDescriptor.Builder() - .name("Batching Max Messages") - .description("Set the maximum number of messages permitted in a batch. default: " - + "1000 If set to a value greater than 1, messages will be queued until this " - + "threshold is reached or batch interval has elapsed") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("1000") - .build(); - - public static final PropertyDescriptor BATCH_INTERVAL = new PropertyDescriptor.Builder() - .name("Batch Interval") - .description("Set the time period within which the messages sent will be batched default: 10ms " - + "if batch messages are enabled. If set to a non zero value, messages will be queued until " - + "this time interval or until the Batching Max Messages threshould has been reached") - .required(false) - .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) - .defaultValue("10") - .build(); - - public static final PropertyDescriptor BLOCK_IF_QUEUE_FULL = new PropertyDescriptor.Builder() - .name("Block if Message Queue Full") - .description("Set whether the processor should block when the outgoing message queue is full. " - + "Default is false. If set to false, send operations will immediately fail with " - + "ProducerQueueIsFullError when there is no space left in pending queue.") - .required(false) - .addValidator(StandardValidators.BOOLEAN_VALIDATOR) - .defaultValue("false") - .build(); - - public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() - .name("Compression Type") - .description("Set the compression type for the producer.") - .required(false) - .allowableValues(COMPRESSION_TYPE_NONE, COMPRESSION_TYPE_LZ4, COMPRESSION_TYPE_ZLIB) - .defaultValue(COMPRESSION_TYPE_NONE.getValue()) - .build(); - - public static final PropertyDescriptor MESSAGE_ROUTING_MODE = new PropertyDescriptor.Builder() - .name("Message Routing Mode") - .description("Set the message routing mode for the producer. This applies only if the destination topic is partitioned") - .required(false) - .allowableValues(MESSAGE_ROUTING_MODE_CUSTOM_PARTITION, MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION, MESSAGE_ROUTING_MODE_SINGLE_PARTITION) - .defaultValue(MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION.getValue()) - .build(); - - public static final PropertyDescriptor PENDING_MAX_MESSAGES = new PropertyDescriptor.Builder() - .name("Max Pending Messages") - .description("Set the max size of the queue holding the messages pending to receive an " - + "acknowledgment from the broker.") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .defaultValue("1000") - .build(); - - private static final List PROPERTIES; - private static final Set RELATIONSHIPS; - - private LRUCache producers; - private ProducerConfiguration producerConfig; - - static { - final List properties = new ArrayList<>(); - properties.add(PULSAR_CLIENT_SERVICE); - properties.add(TOPIC); - properties.add(ASYNC_ENABLED); - properties.add(BATCHING_ENABLED); - properties.add(BATCHING_MAX_MESSAGES); - properties.add(BATCH_INTERVAL); - properties.add(BLOCK_IF_QUEUE_FULL); - properties.add(COMPRESSION_TYPE); - properties.add(MESSAGE_ROUTING_MODE); - properties.add(PENDING_MAX_MESSAGES); - - PROPERTIES = Collections.unmodifiableList(properties); - - final Set relationships = new HashSet<>(); - relationships.add(REL_SUCCESS); - relationships.add(REL_FAILURE); - RELATIONSHIPS = Collections.unmodifiableSet(relationships); - } - - @Override - public Set getRelationships() { - return RELATIONSHIPS; - } - - @Override - protected List getSupportedPropertyDescriptors() { - return PROPERTIES; - } - - @OnStopped - public void cleanUp(final ProcessContext context) { - // Close all of the producers and invalidate them, so they get removed from the Resource Pool - getProducerCache(context).clear(); - } - - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - - FlowFile flowFile = session.get(); - - if (flowFile == null) - return; - - final ComponentLog logger = getLogger(); - final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); - - if (StringUtils.isBlank(topic)) { - logger.error("Invalid topic specified {}", new Object[] {topic}); - session.transfer(flowFile, REL_FAILURE); - return; - } - - // Read the contents of the FlowFile into a byte array - final byte[] messageContent = new byte[(int) flowFile.getSize()]; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - StreamUtils.fillBuffer(in, messageContent, true); - } - }); - - // Nothing to do, so skip this Flow file. - if (messageContent == null || messageContent.length < 1) { - session.transfer(flowFile, REL_SUCCESS); - return; - } - - try { - - Producer producer = getWrappedProducer(topic, context).getProducer(); - - if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { - this.sendAsync(producer, session, flowFile, messageContent); - } else { - this.send(producer, session, flowFile, messageContent); - } - - } catch (final PulsarClientException e) { - logger.error("Failed to connect to Pulsar Server due to {}", new Object[]{e}); - session.penalize(flowFile); - session.transfer(flowFile, REL_FAILURE); - } - - } - - - private void send(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) throws PulsarClientException { - - MessageId msgId = producer.send(messageContent); - - if (msgId != null) { - - flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); - session.adjustCounter("Messages Sent", 1, true); - session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); - session.transfer(flowFile, REL_SUCCESS); - - } else { - session.transfer(flowFile, REL_FAILURE); - } - - } - - private void sendAsync(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) { - - producer.sendAsync(messageContent).handle((msgId, ex) -> { - if (msgId != null) { - return msgId; - } else { - // TODO Communicate the error back up to the onTrigger method so we can invalidate this producer. - getLogger().warn("Problem ", ex); - return null; - } - }); - - flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); - session.adjustCounter("Messages Sent", 1, true); - session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); - session.transfer(flowFile, REL_SUCCESS); - - } - - private PulsarProducer getWrappedProducer(String topic, ProcessContext context) throws PulsarClientException, IllegalArgumentException { - - PulsarProducer producer = getProducerCache(context).get(topic); - - if (producer != null) - return producer; - - try { - producer = context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class) - .getProducerPool().acquire(getProducerProperties(context, topic)); - - if (producer != null) { - producers.put(topic, producer); - } - - return producer; - - } catch (InterruptedException e) { - return null; - } - - } - - private LRUCache getProducerCache(ProcessContext context) { - if (producers == null) { - - ResourcePool pool = context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class) - .getProducerPool(); - - producers = new LRUCache (20, pool); - } - - return producers; - } - - private Properties getProducerProperties(ProcessContext context, String topic) { - - Properties props = new Properties(); - props.put(PulsarProducerFactory.TOPIC_NAME, topic); - props.put(PulsarProducerFactory.PRODUCER_CONFIG, getProducerConfig(context)); - return props; - } - - private ProducerConfiguration getProducerConfig(ProcessContext context) { - - if (producerConfig == null) { - producerConfig = new ProducerConfiguration(); - - if (context.getProperty(BATCHING_ENABLED).isSet()) - producerConfig.setBatchingEnabled(context.getProperty(BATCHING_ENABLED).asBoolean()); - - if (context.getProperty(BATCHING_MAX_MESSAGES).isSet()) - producerConfig.setBatchingMaxMessages(context.getProperty(BATCHING_MAX_MESSAGES).asInteger()); - - if (context.getProperty(BATCH_INTERVAL).isSet()) - producerConfig.setBatchingMaxPublishDelay(context.getProperty(BATCH_INTERVAL).asLong(), TimeUnit.MILLISECONDS); - - if (context.getProperty(BLOCK_IF_QUEUE_FULL).isSet()) - producerConfig.setBlockIfQueueFull(context.getProperty(BLOCK_IF_QUEUE_FULL).asBoolean()); - - if (context.getProperty(COMPRESSION_TYPE).isSet()) - producerConfig.setCompressionType(CompressionType.valueOf(context.getProperty(COMPRESSION_TYPE).getValue())); - - if (context.getProperty(PENDING_MAX_MESSAGES).isSet()) - producerConfig.setMaxPendingMessages(context.getProperty(PENDING_MAX_MESSAGES).asInteger()); - - if (context.getProperty(MESSAGE_ROUTING_MODE).isSet()) - producerConfig.setMessageRoutingMode(MessageRoutingMode.valueOf(context.getProperty(MESSAGE_ROUTING_MODE).getValue())); - } - - return producerConfig; - } - -} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index 16a1d7132811..a05863c23575 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -12,10 +12,6 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -<<<<<<< HEAD org.apache.nifi.processors.pulsar.ConsumePulsar_1_0 org.apache.nifi.processors.pulsar.PublishPulsar_1_0 -======= -org.apache.nifi.processors.pulsar.ConsumePulsar -org.apache.nifi.processors.pulsar.PublishPulsar ->>>>>>> Added Pulsar processors and Controller Service + diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java index 2c61b4a3b396..1bee9dc120a5 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -32,10 +32,6 @@ protected void addPulsarClientService() throws InitializationException { final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); runner.addControllerService("pulsarClient", pulsarClient); runner.enableControllerService(pulsarClient); -<<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.PULSAR_CLIENT_SERVICE, "pulsarClient"); -======= - runner.setProperty(PublishPulsar.PULSAR_CLIENT_SERVICE, "pulsarClient"); ->>>>>>> Added Pulsar processors and Controller Service } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java index 218b4b126ac9..6f138c0442c7 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -49,12 +49,8 @@ public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { -<<<<<<< HEAD runner = TestRunners.newTestRunner(ConsumePulsar_1_0.class); -======= - runner = TestRunners.newTestRunner(ConsumePulsar.class); ->>>>>>> Added Pulsar processors and Controller Service - + mockClient = mock(PulsarClient.class); mockConsumer = mock(Consumer.class); mockMessage = mock(Message.class); @@ -81,6 +77,7 @@ public void init() throws InitializationException { public void emptyMessageTest() { when(mockMessage.getData()).thenReturn("".getBytes()); +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); @@ -92,6 +89,12 @@ public void emptyMessageTest() { runner.run(); runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); + runner.run(); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 } @Test @@ -121,6 +124,7 @@ public void multipleAsyncMessagesTest() throws PulsarClientException { public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); @@ -132,6 +136,12 @@ public void onStoppedTest() throws NoSuchMethodException, SecurityException, Pul runner.run(10, true); runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); + runner.run(10, true); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 runner.assertQueueEmpty(); @@ -150,6 +160,7 @@ private void sendMessages(String msg, String topic, String sub, boolean async, i when(mockMessage.getData()).thenReturn(msg.getBytes()); +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); @@ -163,12 +174,21 @@ private void sendMessages(String msg, String topic, String sub, boolean async, i runner.setProperty(ConsumePulsar.ASYNC_ENABLED, Boolean.toString(async)); runner.setProperty(ConsumePulsar.TOPIC, topic); runner.setProperty(ConsumePulsar.SUBSCRIPTION, sub); +======= + runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 runner.run(itertions, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); +<<<<<<< HEAD List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar.REL_SUCCESS); >>>>>>> Added Pulsar processors and Controller Service +======= + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_0.REL_SUCCESS); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 assertEquals(itertions, flowFiles.size()); for (MockFlowFile ff : flowFiles) { diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java index 754f47fe1d0e..67b738710a4b 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -44,12 +44,9 @@ public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { -<<<<<<< HEAD + runner = TestRunners.newTestRunner(PublishPulsar_1_0.class); -======= - runner = TestRunners.newTestRunner(PublishPulsar.class); ->>>>>>> Added Pulsar processors and Controller Service - + mockClient = mock(PulsarClient.class); mockProducer = mock(Producer.class); @@ -86,6 +83,7 @@ public boolean matches(Object argument) { @Test public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); @@ -98,15 +96,22 @@ public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClient runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); ======= runner.setProperty(PublishPulsar.TOPIC, "${topic}"); +======= + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 final String content = "some content"; Map attributes = new HashMap (); - attributes.put(PublishPulsar.TOPIC.getName(), ""); + attributes.put(PublishPulsar_1_0.TOPIC.getName(), ""); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); +<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar.REL_FAILURE); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 // Confirm that no Producer as created verify(mockClient, times(0)).createProducer(anyString()); @@ -115,6 +120,7 @@ public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); @@ -127,15 +133,22 @@ public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClient runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ======= runner.setProperty(PublishPulsar.TOPIC, "${topic}"); +======= + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 final String content = "some content"; Map attributes = new HashMap (); - attributes.put(PublishPulsar.TOPIC.getName(), "topic-b"); + attributes.put(PublishPulsar_1_0.TOPIC.getName(), "topic-b"); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); +<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 // Verify that we sent the data to topic-b. verify(mockClient, times(1)).createProducer("topic-b"); @@ -144,15 +157,20 @@ public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); ======= runner.setProperty(PublishPulsar.TOPIC, "my-topic"); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD <<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); @@ -162,6 +180,11 @@ public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClie final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -174,6 +197,7 @@ public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClie @Test public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); @@ -181,10 +205,15 @@ public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, Pulsa runner.setProperty(PublishPulsar.TOPIC, "my-topic"); runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD <<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); @@ -194,6 +223,11 @@ public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, Pulsa final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -206,22 +240,30 @@ public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, Pulsa @Test public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); ======= runner.setProperty(PublishPulsar.TOPIC, "my-topic"); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD <<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ======= runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 } // Verify that the send method on the producer was called with the expected content @@ -231,6 +273,7 @@ public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarC @Test public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); @@ -238,17 +281,25 @@ public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, Pu runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD <<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ======= runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 } // Verify that the send method on the producer was called with the expected content @@ -257,6 +308,7 @@ public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, Pu @Test public void stressTest() throws UnsupportedEncodingException { +<<<<<<< HEAD <<<<<<< HEAD runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); @@ -264,16 +316,24 @@ public void stressTest() throws UnsupportedEncodingException { runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 final String content = "some content"; for (int idx = 0; idx < 9999; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); +<<<<<<< HEAD <<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ======= runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); >>>>>>> Added Pulsar processors and Controller Service +======= + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); +>>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 } } From f36b0b61680c55a5ac0279a44cdc580b42f20118 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 9 Mar 2018 19:01:06 -0800 Subject: [PATCH 10/16] Combined Pulsar components into a single bundle --- .../nifi-pulsar-client-service-api/pom.xml | 40 ++ .../apache/nifi/pulsar/PulsarClientPool.java | 33 ++ .../apache/nifi/pulsar/PulsarConsumer.java | 68 +++ .../apache/nifi/pulsar/PulsarProducer.java | 63 +++ .../apache/nifi/pulsar/cache/LRUCache.java | 68 +++ .../nifi/pulsar/pool/PoolableResource.java | 25 + .../pulsar/pool/PulsarConsumerFactory.java | 56 +++ .../pulsar/pool/PulsarProducerFactory.java | 51 ++ .../pool/ResourceCreationException.java | 31 ++ .../pulsar/pool/ResourceExceptionHandler.java | 23 + .../nifi/pulsar/pool/ResourceFactory.java | 24 + .../apache/nifi/pulsar/pool/ResourcePool.java | 38 ++ .../nifi/pulsar/cache/LRUCacheTest.java | 136 +++++ .../nifi-pulsar-client-service/pom.xml | 64 +++ .../nifi/pulsar/StandardPulsarClientPool.java | 300 +++++++++++ .../pool/ResourceExceptionHandlerImpl.java | 31 ++ .../nifi/pulsar/pool/ResourcePoolImpl.java | 138 +++++ ...g.apache.nifi.controller.ControllerService | 15 + .../org/apache/nifi/pulsar/TestProcessor.java | 45 ++ .../TestStandardPulsarClientService.java | 44 ++ .../pulsar/pool/MockPulsarClientService.java | 148 ++++++ .../nifi/pulsar/pool/ResourcePoolTest.java | 146 ++++++ .../pulsar/AbstractPulsarProcessor.java | 30 +- .../processors/pulsar/ConsumePulsar_1_0.java | 470 +++++++++--------- .../processors/pulsar/PublishPulsar_1_0.java | 439 ++++++++-------- .../pulsar/AbstractPulsarProcessorTest.java | 10 +- .../pulsar/ConsumePulsarProcessorTest.java | 190 +++---- .../pulsar/MockPulsarClientService.java | 227 ++++----- .../pulsar/PublishPulsarProcessorTest.java | 374 +++++--------- nifi-nar-bundles/nifi-pulsar-bundle/pom.xml | 4 + nifi-nar-bundles/pom.xml | 3 + 31 files changed, 2377 insertions(+), 957 deletions(-) create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml new file mode 100644 index 000000000000..2259fc94f5ff --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.6.0-SNAPSHOT + + + nifi-pulsar-client-service-api + jar + + + + org.apache.nifi + nifi-api + provided + + + org.apache.pulsar + pulsar-client + 1.21.0-incubating + + + diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java new file mode 100644 index 000000000000..abd51f5b03f8 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java @@ -0,0 +1,33 @@ +/* + * 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.pulsar; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.pulsar.pool.ResourcePool; + + +@Tags({"Pulsar"}) +@CapabilityDescription("Provides the ability to create Pulsar Producer / Consumer instances on demand, based on the configuration." + + "properties defined") +public interface PulsarClientPool extends ControllerService { + + public ResourcePool getProducerPool(); + + public ResourcePool getConsumerPool(); +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java new file mode 100644 index 000000000000..34f7453637ad --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java @@ -0,0 +1,68 @@ +/* + * 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.pulsar; + +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.ConsumerStats; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("unused") +public class PulsarConsumer implements PoolableResource { + + private static final Logger logger = LoggerFactory.getLogger(PulsarConsumer.class); + + private final Consumer consumer; + private final String topicName; + private final String subscriptionName; + private boolean closed = false; + + public PulsarConsumer(Consumer consumer, String topic, String subscription) throws PulsarClientException { + this.consumer = consumer; + this.topicName = topic; + this.subscriptionName = subscription; + } + + public void close() { + + logger.info("Closing consumer for topic {} and subscription {}", new Object[] {topicName, subscriptionName}); + closed = true; + + try { + consumer.unsubscribe(); + consumer.close(); + } catch (PulsarClientException e) { + logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); + closed = false; + } + } + + public boolean isClosed() { + return closed; + } + + public Consumer getConsumer() { + return this.consumer; + } + + public ConsumerStats getStats() { + return this.consumer.getStats(); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java new file mode 100644 index 000000000000..ea335f2c4e70 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarProducer.java @@ -0,0 +1,63 @@ +/* + * 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.pulsar; + +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PulsarProducer implements PoolableResource { + + private static final Logger logger = LoggerFactory.getLogger(PulsarProducer.class); + + private final Producer producer; + private boolean closed = false; + private final String topicName; + + public PulsarProducer(Producer producer, String topicName) throws PulsarClientException { + this.topicName = topicName; + this.producer = producer; + } + + public Producer getProducer() { + return producer; + } + + public String getName() { + return topicName; + } + + public boolean isClosed() { + return this.closed; + } + + public void close() { + + logger.info("Closing producer for topic {} ", new Object[] {topicName}); + + this.closed = true; + try { + producer.close(); + } catch (PulsarClientException e) { + logger.error("Unable to close connection to Pulsar due to {}; resources may not be cleaned up appropriately", e); + closed = false; + } + + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java new file mode 100644 index 000000000000..0748d8a2ca8a --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java @@ -0,0 +1,68 @@ +/* + * 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.pulsar.cache; + +import java.util.LinkedHashMap; + +import org.apache.nifi.pulsar.pool.PoolableResource; +import org.apache.nifi.pulsar.pool.ResourcePool; + +public class LRUCache { + + private LinkedHashMap lruCacheMap; + private final int capacity; + private final boolean SORT_BY_ACCESS = true; + private final float LOAD_FACTOR = 0.75F; + private final ResourcePool resourcePool; + + public LRUCache(int capacity, ResourcePool resourcePool){ + this.capacity = capacity; + this.lruCacheMap = new LinkedHashMap<>(capacity, LOAD_FACTOR, SORT_BY_ACCESS); + this.resourcePool = resourcePool; + } + + public V get(K k){ + return lruCacheMap.get(k); + } + + public void put(K k, V v){ + if(lruCacheMap.containsKey(k)){ + lruCacheMap.remove(k); + } else if(lruCacheMap.size() >= capacity){ + K victimKey = lruCacheMap.keySet().iterator().next(); + V victim = lruCacheMap.get(victimKey); + lruCacheMap.remove(victimKey); + resourcePool.evict(victim); + } + lruCacheMap.put(k, v); + } + + public int getSize() { + return lruCacheMap.size(); + } + + public void clear() { + for (V victim :lruCacheMap.values()) { + resourcePool.evict(victim); + } + lruCacheMap.clear(); + } + + public void printSequence() { + System.out.println(lruCacheMap.keySet()); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java new file mode 100644 index 000000000000..4042e808b7e5 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java @@ -0,0 +1,25 @@ +/* + * 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.pulsar.pool; + +public interface PoolableResource { + + public void close(); + + public boolean isClosed(); + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java new file mode 100644 index 000000000000..5101c5f4b2ad --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java @@ -0,0 +1,56 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.pulsar.client.api.ConsumerConfiguration; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; + +public class PulsarConsumerFactory implements ResourceFactory { + + public static final String TOPIC_NAME = "topic"; + public static final String SUBSCRIPTION_NAME = "subscription"; + public static final String CONSUMER_CONFIG = "Consumer-Configuration"; + + private PulsarClient client; + + public PulsarConsumerFactory(PulsarClient client) { + this.client = client; + } + + @Override + public PulsarConsumer create(Properties props) throws ResourceCreationException { + + String topic = props.getProperty(TOPIC_NAME); + String subscription = props.getProperty(SUBSCRIPTION_NAME); + ConsumerConfiguration config = (ConsumerConfiguration) props.get(CONSUMER_CONFIG); + + try { + // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice + return (config == null) ? new PulsarConsumer(client.subscribe(topic, subscription), topic, subscription) : + new PulsarConsumer(client.subscribe(topic, subscription, config), topic, subscription); + + } catch (PulsarClientException e) { + throw new ResourceCreationException(e); + } + + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java new file mode 100644 index 000000000000..9acb15fc0ea7 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarProducerFactory.java @@ -0,0 +1,51 @@ +/* + * 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.pulsar.pool; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.pulsar.client.api.ProducerConfiguration; +import org.apache.pulsar.client.api.PulsarClient; + +public class PulsarProducerFactory implements ResourceFactory { + + public static final String TOPIC_NAME = "topic"; + public static final String PRODUCER_CONFIG = "Producer-Configuration"; + + private PulsarClient client; + + public PulsarProducerFactory(PulsarClient client) { + this.client = client; + } + + @Override + public PulsarProducer create(Properties props) throws ResourceCreationException { + + String topic = props.getProperty(TOPIC_NAME); + ProducerConfiguration config = (ProducerConfiguration) props.get(PRODUCER_CONFIG); + + try { + // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice + return (config == null) ? new PulsarProducer(client.createProducer(topic), topic) : + new PulsarProducer(client.createProducer(topic, config), topic); + } catch (Exception e) { + throw new ResourceCreationException(e); + } + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java new file mode 100644 index 000000000000..9ba77287f7ef --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceCreationException.java @@ -0,0 +1,31 @@ +/* + * 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.pulsar.pool; + +public class ResourceCreationException extends Exception { + + private static final long serialVersionUID = -1796731800483756134L; + + public ResourceCreationException(String s) { + super(s); + } + + public ResourceCreationException(Exception e) { + super(e); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java new file mode 100644 index 000000000000..f116f257896b --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java @@ -0,0 +1,23 @@ + +/* + * 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.pulsar.pool; + +public interface ResourceExceptionHandler { + + void handle(Exception exc); +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java new file mode 100644 index 000000000000..9ed21750c812 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.pulsar.pool; + +import java.util.Properties; + +public interface ResourceFactory { + + public R create(Properties props) throws ResourceCreationException; +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java new file mode 100644 index 000000000000..670e56289bc1 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourcePool.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.pulsar.pool; + +import java.util.Properties; + +public interface ResourcePool { + + /** + * Acquire a resource from the pool. Creating one if necessary + */ + public R acquire(Properties props) throws InterruptedException; + + /** + * Evict the resource from the pool, destroying it. + * Call this method is the resource is known to be in an unusable state. + */ + public void evict(R resource); + + /** + * Place the resource back into the pool for future use. + */ + public void release(R resource); +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java new file mode 100644 index 000000000000..081618f748b1 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java @@ -0,0 +1,136 @@ +/* + * 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.pulsar.cache; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class LRUCacheTest { + + @Mock + private ResourcePool mockedResourcePool; + + @Mock + private PulsarProducer mockedPulsarProducer; + + @SuppressWarnings("unchecked") + @Before + public void setUp() throws InterruptedException { + mockedResourcePool = mock(ResourcePool.class); + mockedPulsarProducer = mock(PulsarProducer.class); + when(mockedResourcePool.acquire(any(Properties.class))).thenReturn(mockedPulsarProducer); + } + + /** + * Make sure the LRUCache functions as a Map + */ + @Test + public void simpleTest() { + LRUCache cache = new LRUCache(10, mockedResourcePool); + + for (Character i='A'; i<='E'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + assertEquals(5, cache.getSize()); + + for (Character i='A'; i<='E'; i++){ + assertNotNull( cache.get(i.toString())); + } + } + + @Test + public void evictionTest() { + + LRUCache cache = new LRUCache(5, mockedResourcePool); + + for (Character i='A'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + // Make sure we only have 5 items in the cache + assertEquals(5, cache.getSize()); + + // Make sure we have the last 5 items added to the cache + for (Character i='V'; i<='Z'; i++){ + assertNotNull( cache.get(i.toString())); + } + + // Make sure the evict method on the resource pool was called 21 times, once for every item removed. + verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); + } + + @Test + public void evictionLruTest() { + + LRUCache cache = new LRUCache(5, mockedResourcePool); + + final Character A = 'A'; + + // Write 25 items to the cache, and the letter 'A' every other put. + for (Character i='B'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + cache.put(A.toString(), mockedPulsarProducer); + } + + // Make sure we only have 5 items in the cache + assertEquals(5, cache.getSize()); + + // Make sure that the letter 'A' is still in the cache due to frequent access + assertNotNull( cache.get(A.toString()) ); + + // Make sure we have the last 4 items added to the cache + for (Character i='W'; i<='Z'; i++){ + assertNotNull( cache.get(i.toString())); + } + + // Make sure the evict method on the resource pool was called 21 times, once for every item removed. + verify(mockedResourcePool, times(21)).evict(mockedPulsarProducer); + } + + @Test + public void clearTest() { + LRUCache cache = new LRUCache(26, mockedResourcePool); + + for (Character i='A'; i<='Z'; i++){ + cache.put(i.toString(), mockedPulsarProducer); + } + + // Make sure we only have all the items in the cache + assertEquals(26, cache.getSize()); + cache.clear(); + + // Make sure all the items were removed + assertEquals(0, cache.getSize()); + + // Make sure all the items were evicted from the underlying resource pool + verify(mockedResourcePool, times(26)).evict(mockedPulsarProducer); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml new file mode 100644 index 000000000000..5c3d3a6efc49 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml @@ -0,0 +1,64 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-client-services + 1.6.0-SNAPSHOT + + + nifi-pulsar-client-service + jar + + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.6.0-SNAPSHOT + + + org.apache.nifi + nifi-api + provided + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-ssl-context-service-api + provided + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java new file mode 100644 index 000000000000..7502b220f7cc --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java @@ -0,0 +1,300 @@ +/* + * 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.pulsar; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.nifi.pulsar.pool.ResourcePoolImpl; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.pulsar.client.api.ClientConfiguration; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; + +@Tags({ "Pulsar"}) +@CapabilityDescription("Standard ControllerService implementation of PulsarClientService.") +public class StandardPulsarClientPool extends AbstractControllerService implements PulsarClientPool { + + public static final PropertyDescriptor PULSAR_SERVICE_URL = new PropertyDescriptor + .Builder().name("PULSAR_SERVICE_URL") + .displayName("Pulsar Service URL") + .description("URL for the Pulsar cluster, e.g localhost:6650") + .required(true) + .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR) + .build(); + + public static final PropertyDescriptor CONCURRENT_LOOKUP_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum concurrent lookup-requests") + .description("Number of concurrent lookup-requests allowed on each broker-connection to prevent " + + "overload on broker. (default: 5000) It should be configured with higher value only in case " + + "of it requires to produce/subscribe on thousands of topics") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("5000") + .build(); + + public static final PropertyDescriptor CONNECTIONS_PER_BROKER = new PropertyDescriptor.Builder() + .name("Maximum connects per Pulsar broker") + .description("Sets the max number of connection that the client library will open to a single broker.\n" + + "By default, the connection pool will use a single connection for all the producers and consumers. " + + "Increasing this parameter may improve throughput when using many producers over a high latency connection") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor IO_THREADS = new PropertyDescriptor.Builder() + .name("I/O Threads") + .description("The number of threads to be used for handling connections to brokers (default: 1 thread)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor LISTENER_THREADS = new PropertyDescriptor.Builder() + .name("Listener Threads") + .description("The number of threads to be used for message listeners (default: 1 thread)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAXIMUM_REJECTED_REQUESTS = new PropertyDescriptor.Builder() + .name("Maximum rejected requests per connection") + .description("Max number of broker-rejected requests in a certain time-frame (30 seconds) after " + + "which current connection will be closed and client creates a new connection that give " + + "chance to connect a different broker (default: 50)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("50") + .build(); + + public static final PropertyDescriptor OPERATION_TIMEOUT = new PropertyDescriptor.Builder() + .name("Operation Timeout") + .description("Producer-create, subscribe and unsubscribe operations will be retried until this " + + "interval, after which the operation will be maked as failed (default: 30 seconds)") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("30") + .build(); + + public static final PropertyDescriptor STATS_INTERVAL = new PropertyDescriptor.Builder() + .name("Stats interval") + .description("The interval between each stat info (default: 60 seconds) Stats will be activated " + + "with positive statsIntervalSeconds It should be set to at least 1 second") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("60") + .build(); + + public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder() + .name("Use TCP nodelay flag") + .description("Configure whether to use TCP no-delay flag on the connection, to disable Nagle algorithm.\n" + + "No-delay features make sure packets are sent out on the network as soon as possible, and it's critical " + + "to achieve low latency publishes. On the other hand, sending out a huge number of small packets might " + + "limit the overall throughput, so if latency is not a concern, it's advisable to set the useTcpNoDelay " + + "flag to false.") + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + + public static final PropertyDescriptor MAX_PRODUCERS = new PropertyDescriptor + .Builder().name("MAX_PRODUCERS") + .displayName("Producer Pool Size") + .description("The Maximum Number of Pulsar Producers created by this Pulsar Client Pool") + .required(true) + .defaultValue("10") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final PropertyDescriptor MAX_CONSUMERS = new PropertyDescriptor + .Builder().name("MAX_CONSUMERS") + .displayName("Consumer Pool Size") + .description("The Maximum Number of Pulsar consumers created by this Pulsar Client Pool") + .required(true) + .defaultValue("10") + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .build(); + + public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("ssl.context.service") + .displayName("SSL Context Service") + .description("Specifies the SSL Context Service to use for communicating with Pulsar.") + .required(false) + .identifiesControllerService(SSLContextService.class) + .build(); + + private static final List properties; + private volatile PulsarClient client; + + private volatile ResourcePoolImpl producers; + private volatile ResourcePoolImpl consumers; + private ClientConfiguration clientConfig; + + static { + final List props = new ArrayList<>(); + props.add(PULSAR_SERVICE_URL); + props.add(MAX_CONSUMERS); + props.add(MAX_PRODUCERS); + props.add(CONCURRENT_LOOKUP_REQUESTS); + props.add(CONNECTIONS_PER_BROKER); + props.add(IO_THREADS); + props.add(LISTENER_THREADS); + props.add(MAXIMUM_REJECTED_REQUESTS); + props.add(OPERATION_TIMEOUT); + props.add(STATS_INTERVAL); + props.add(USE_TCP_NO_DELAY); + properties = Collections.unmodifiableList(props); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + /** + * @param context + * the configuration context + * @throws InitializationException + * if unable to create a database connection + */ + @OnEnabled + public void onEnabled(final ConfigurationContext context) throws InitializationException { + + createClient(context); + + if (this.client == null) + throw new InitializationException("Unable to create Pulsar Client"); + + producers = new ResourcePoolImpl(new PulsarProducerFactory(client), context.getProperty(MAX_PRODUCERS).asInteger()); + consumers = new ResourcePoolImpl(new PulsarConsumerFactory(client), context.getProperty(MAX_CONSUMERS).asInteger()); + + } + + private void createClient(final ConfigurationContext context) throws InitializationException { + + // We can't create a client without a service URL. + if (!context.getProperty(PULSAR_SERVICE_URL).isSet()) { + return; + } + + try { + this.client = PulsarClient.create(buildPulsarBrokerRootUrl(context.getProperty(PULSAR_SERVICE_URL).getValue(), + getClientConfig(context).isUseTls()), getClientConfig(context)); + + } catch (Exception e) { + throw new InitializationException("Unable to create Pulsar Client", e); + } + + } + + private static String buildPulsarBrokerRootUrl(String uri, boolean tlsEnabled) { + StringBuilder builder = new StringBuilder(); + builder.append("pulsar"); + + if (tlsEnabled) + builder.append("+ssl"); + + builder.append("://"); + builder.append(uri); + return builder.toString(); + } + + private ClientConfiguration getClientConfig(ConfigurationContext context) throws UnsupportedAuthenticationException { + + if (clientConfig == null) { + clientConfig = new ClientConfiguration(); + + if (context.getProperty(CONCURRENT_LOOKUP_REQUESTS).isSet()) { + clientConfig.setConcurrentLookupRequest(context.getProperty(CONCURRENT_LOOKUP_REQUESTS).asInteger()); + } + + if (context.getProperty(CONNECTIONS_PER_BROKER).isSet()) { + clientConfig.setConnectionsPerBroker(context.getProperty(CONNECTIONS_PER_BROKER).asInteger()); + } + + if (context.getProperty(IO_THREADS).isSet()) { + clientConfig.setIoThreads(context.getProperty(IO_THREADS).asInteger()); + } + + if (context.getProperty(LISTENER_THREADS).isSet()) { + clientConfig.setListenerThreads(context.getProperty(LISTENER_THREADS).asInteger()); + } + + if (context.getProperty(MAXIMUM_REJECTED_REQUESTS).isSet()) { + clientConfig.setMaxNumberOfRejectedRequestPerConnection(context.getProperty(MAXIMUM_REJECTED_REQUESTS).asInteger()); + } + + if (context.getProperty(OPERATION_TIMEOUT).isSet()) { + clientConfig.setOperationTimeout(context.getProperty(OPERATION_TIMEOUT).asInteger(), TimeUnit.SECONDS); + } + + if (context.getProperty(STATS_INTERVAL).isSet()) { + clientConfig.setStatsInterval(context.getProperty(STATS_INTERVAL).asLong(), TimeUnit.SECONDS); + } + + if (context.getProperty(USE_TCP_NO_DELAY).isSet()) { + clientConfig.setUseTcpNoDelay(context.getProperty(USE_TCP_NO_DELAY).asBoolean()); + } + + // Configure TLS + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + + if (sslContextService != null && sslContextService.isTrustStoreConfigured() && sslContextService.isKeyStoreConfigured()) { + clientConfig.setUseTls(true); + clientConfig.setTlsTrustCertsFilePath(sslContextService.getTrustStoreFile()); + + Map authParams = new HashMap<>(); + + // TODO This should be a different value than the TlsTrustCertsFilePath above. + authParams.put("tlsCertFile", sslContextService.getTrustStoreFile()); + authParams.put("tlsKeyFile", sslContextService.getKeyStoreFile()); + clientConfig.setAuthentication(AuthenticationTls.class.getName(), authParams); + } + } + + return clientConfig; + } + + @Override + public ResourcePool getProducerPool() { + return this.producers; + } + + @Override + public ResourcePool getConsumerPool() { + return this.consumers; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java new file mode 100644 index 000000000000..edb26003fec8 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandlerImpl.java @@ -0,0 +1,31 @@ +/* + * 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.pulsar.pool; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ResourceExceptionHandlerImpl implements ResourceExceptionHandler { + + private static Logger logger = LoggerFactory.getLogger(ResourceExceptionHandlerImpl.class); + + @Override + public void handle(Exception exc) { + logger.error("Unable to create Resource", exc); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java new file mode 100644 index 000000000000..441879630f0d --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java @@ -0,0 +1,138 @@ +/* + * 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.pulsar.pool; + +import java.util.Iterator; +import java.util.Properties; +import java.util.Vector; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + + +public class ResourcePoolImpl implements ResourcePool { + + private final Lock lock = new ReentrantLock(); + private final Condition poolAvailable = lock.newCondition(); + private int max_resources; + private final Vector pool; + + private final ResourceExceptionHandler resourceExceptionHandler; + private final ResourceFactory factory; + + public ResourcePoolImpl(ResourceFactory factory, int max_resources) { + this(factory, new ResourceExceptionHandlerImpl(), max_resources); + } + + public ResourcePoolImpl(ResourceFactory factory, ResourceExceptionHandler handler, int max_resources) { + lock.lock(); + try { + this.factory = factory; + this.resourceExceptionHandler = handler; + this.max_resources = max_resources; + this.pool = new Vector(max_resources); + } finally { + lock.unlock(); + } + } + + private R createResource(Properties props) { + R resource = null; + try { + + resource = factory.create(props); + + if (resource == null) + throw new ResourceCreationException("Unable to create resource"); + + } catch (Exception e) { + resourceExceptionHandler.handle(e); + } + return resource; + } + + + /* + * Shutdown the pool and release the resources + */ + public void close() { + + Iterator itr = pool.iterator(); + while (itr.hasNext()) { + itr.next().close(); + } + + } + + public boolean isEmpty() { + return (pool.isEmpty()); + } + + public boolean isFull() { + return (pool != null && pool.size() == max_resources); + } + + @Override + public R acquire(Properties props) throws InterruptedException { + lock.lock(); + try { + while (max_resources <= 0) { + poolAvailable.await(); + } + + --max_resources; + + if (pool != null) { + int size = pool.size(); + if (size > 0) + return pool.remove(size - 1); + } + return createResource(props); + } finally { + lock.unlock(); + } + } + + @Override + public void evict(R resource) { + lock.lock(); + try { + + // Attempt to close the connection + if (!resource.isClosed()) + resource.close(); + + pool.removeElement(resource); + --max_resources; + poolAvailable.signal(); + } finally { + lock.unlock(); + } + } + + @Override + public void release(R resource) { + lock.lock(); + try { + pool.addElement(resource); + ++max_resources; + poolAvailable.signal(); + } finally { + lock.unlock(); + } + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 000000000000..c1eecc70a562 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,15 @@ +# 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. +org.apache.nifi.pulsar.StandardPulsarClientPool \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java new file mode 100644 index 000000000000..72d17006e83f --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestProcessor.java @@ -0,0 +1,45 @@ +/* + * 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.pulsar; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +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; + +public class TestProcessor extends AbstractProcessor { + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + } + + @Override + protected List getSupportedPropertyDescriptors() { + List propDescs = new ArrayList<>(); + propDescs.add(new PropertyDescriptor.Builder() + .name("StandardPulsarClientService test processor") + .description("StandardPulsarClientService test processor") + .identifiesControllerService(StandardPulsarClientPool.class) + .required(true) + .build()); + return propDescs; + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java new file mode 100644 index 000000000000..4b9982df30db --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java @@ -0,0 +1,44 @@ +/* + * 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.pulsar; + +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class TestStandardPulsarClientService { + + @Before + public void init() { + + } + + @Test + public void testService() throws InitializationException { + final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class); + final PulsarClientPool service = new StandardPulsarClientPool(); + runner.addControllerService("test-good", service); + + runner.setProperty(service, StandardPulsarClientPool.PULSAR_SERVICE_URL, "localhost:6667"); + // runner.enableControllerService(service); + + runner.assertValid(service); + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java new file mode 100644 index 000000000000..2a3a5517bd25 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java @@ -0,0 +1,148 @@ +/* + * 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.pulsar.pool; + +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.pulsar.PulsarClientPool; +import org.apache.nifi.pulsar.PulsarConsumer; +import org.apache.nifi.pulsar.PulsarProducer; +import org.apache.nifi.pulsar.pool.PulsarConsumerFactory; +import org.apache.nifi.pulsar.pool.PulsarProducerFactory; +import org.apache.nifi.pulsar.pool.ResourcePool; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.Rule; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.stubbing.Answer; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.Properties; + +public class MockPulsarClientService extends AbstractControllerService implements PulsarClientPool { + + @Mock + PulsarClient mockClient; + + @Mock + ResourcePool mockProducerPool; + + @Mock + ResourcePool mockConsumerPool; + + @Mock + Producer mockProducer; + + @Mock + Consumer mockConsumer; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @SuppressWarnings("unchecked") + public MockPulsarClientService() { + this.mockClient = mock(PulsarClient.class); + mockProducerPool = mock(ResourcePool.class); + mockConsumerPool = mock(ResourcePool.class); + mockProducer = mock(Producer.class); + mockConsumer = mock(Consumer.class); + + try { + when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( + new Answer() { + @Override + public PulsarProducer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getProducer(props); + } + } + ); + + when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( + new Answer() { + @Override + public PulsarConsumer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getConsumer(props); + } + } + ); + } catch (InterruptedException ex) { + + } + + try { + when(mockClient.createProducer(anyString())).thenReturn(getMockProducer()); + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(null); + } catch (PulsarClientException e) { + e.printStackTrace(); + } + } + + public Producer getMockProducer() { + return mockProducer; + } + + public PulsarClient getMockClient() { + return mockClient; + } + + public PulsarProducer getProducer(Properties props) { + String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); + try { + return new PulsarProducer(mockClient.createProducer(topic), topic); + } catch (PulsarClientException e) { + return null; + } + } + + public PulsarConsumer getConsumer(Properties props) { + String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); + String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); + try { + return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + } catch (PulsarClientException e) { + return null; + } + } + + @Override + public ResourcePool getProducerPool() { + return mockProducerPool; + } + + + @Override + public ResourcePool getConsumerPool() { + return mockConsumerPool; + } + +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java new file mode 100644 index 000000000000..c7c10f12a056 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/ResourcePoolTest.java @@ -0,0 +1,146 @@ +/* + * 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.pulsar.pool; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.Properties; + +import org.apache.nifi.pulsar.PulsarProducer; +import org.junit.Test; + +public class ResourcePoolTest { + + private MockPulsarClientService pulsarClient = new MockPulsarClientService(); + + /*Checks the number of the resources in the Resource Table. + Resource is created only if needed.*/ + @Test + public void testIsEmptyBeforeResourceAcquired() throws Exception { + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + assertTrue(resPool.isEmpty()); + } + + /* Checks if the number of resources in the Resource Table is equal to the + * maximum number of resources declared at the Resource Pool creation time.*/ + @Test + public void testIsFull() throws Exception { + + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + } + + /* + * Checks to see if resources are left in the pool after they + * have been acquired, and then released. + */ + @Test + public void testResourcesReused() throws InterruptedException { + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + for (int i = 0; i <= 4; i++) { + resPool.release(res[i]); + } + + assertFalse(resPool.isEmpty()); + } + + /* + * Checks to see if resources that are invalidated are removed from + * the pool. + */ + @Test + public void testResourcesReleased() throws InterruptedException { + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "topic-a"); + + ResourcePoolImpl resPool = getResourcePool("topic-a", 5); + + PulsarProducer[] res = new PulsarProducer[5]; + for (int i = 0; i <= 4; i++) { + res[i] = resPool.acquire(props); + } + + assertTrue(resPool.isFull()); + + for (int i = 0; i <= 4; i++) { + resPool.evict(res[i]); + } + + assertTrue(resPool.isEmpty()); + } + + @Test + public void testAcquireBlocksWhenEmpty() throws Exception { + + Properties props = new Properties(); + props.setProperty(PulsarProducerFactory.TOPIC_NAME, "A"); + + final ResourcePoolImpl resPool = new ResourcePoolImpl( + new PulsarProducerFactory(pulsarClient.getMockClient()) , 0); + + Thread resourceConsumer = new Thread() { + public void run() { + try { + @SuppressWarnings("unused") + PulsarProducer unused = resPool.acquire(props); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + fail(); // error if control flow reaches this line + } + }; + + resourceConsumer.start(); + Thread.sleep(1000); // waits for the resourceConsumer to block + resourceConsumer.interrupt(); + resourceConsumer.join(1000); // resume after the resourceConsumer ends + assertFalse(resourceConsumer.isAlive()); + + } + + + private ResourcePoolImpl getResourcePool(String topic, int size) { + return new ResourcePoolImpl( + new PulsarProducerFactory(pulsarClient.getMockClient()), size); + } +} diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java index a2ecf430ba0f..508b2558fc2c 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java @@ -23,21 +23,21 @@ import org.apache.nifi.pulsar.PulsarClientPool; public abstract class AbstractPulsarProcessor extends AbstractProcessor { - - protected static final PropertyDescriptor PULSAR_CLIENT_SERVICE = new PropertyDescriptor.Builder() - .name("Pulsar Client Service") - .description("Specified the Pulsar Client Service that can be used to create Pulsar connections") - .required(true) - .identifiesControllerService(PulsarClientPool.class) - .build(); - protected static final Relationship REL_SUCCESS = new Relationship.Builder() - .name("success") - .description("FlowFiles for which all content was sent to Pulsar.") - .build(); + protected static final PropertyDescriptor PULSAR_CLIENT_SERVICE = new PropertyDescriptor.Builder() + .name("Pulsar Client Service") + .description("Specified the Pulsar Client Service that can be used to create Pulsar connections") + .required(true) + .identifiesControllerService(PulsarClientPool.class) + .build(); - protected static final Relationship REL_FAILURE = new Relationship.Builder() - .name("failure") - .description("Any FlowFile that cannot be sent to Pulsar will be routed to this Relationship") - .build(); + protected static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles for which all content was sent to Pulsar.") + .build(); + + protected static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("Any FlowFile that cannot be sent to Pulsar will be routed to this Relationship") + .build(); } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java index f331b11d4944..e67641453a10 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java @@ -59,83 +59,83 @@ + "The complementary NiFi processor for sending messages is PublishPulsar.") @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) public class ConsumePulsar_1_0 extends AbstractPulsarProcessor { - - static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); - static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); - static final AllowableValue FAILOVER = new AllowableValue("Failover", "Failover", "Multiple consumer will be able to use the same subscription name but only 1 consumer " - + "will receive the messages. If that consumer disconnects, one of the other connected consumers will start receiving messages"); - - protected static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() - .name("topic") - .displayName("Topic Name") - .description("The name of the Pulsar Topic.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - static final PropertyDescriptor SUBSCRIPTION = new PropertyDescriptor.Builder() - .name("Subscription") - .displayName("Subscription Name") - .description("The name of the Pulsar subscription to consume from.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + + static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); + static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); + static final AllowableValue FAILOVER = new AllowableValue("Failover", "Failover", "Multiple consumer will be able to use the same subscription name but only 1 consumer " + + "will receive the messages. If that consumer disconnects, one of the other connected consumers will start receiving messages"); + + protected static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + static final PropertyDescriptor SUBSCRIPTION = new PropertyDescriptor.Builder() + .name("Subscription") + .displayName("Subscription Name") + .description("The name of the Pulsar subscription to consume from.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() .name("Async Enabled") .description("Control whether the messages will be consumed asyncronously or not. Messages consumed" - + " syncronously will be acknowledged immediately before processing the next message, while" - + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + + " syncronously will be acknowledged immediately before processing the netX message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") .required(true) .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); - - public static final PropertyDescriptor MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder() .name("Maximum Async Requests") .description("The maximum number of outstanding asynchronous consumer requests for this processor. " - + "Each asynchronous call requires memory, so avoid setting this value to high.") + + "Each asynchronous call requires memory, so avoid setting this value to high.") .required(false) .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .defaultValue("50") .build(); - - public static final PropertyDescriptor ACK_TIMEOUT = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor ACK_TIMEOUT = new PropertyDescriptor.Builder() .name("Acknowledgment Timeout") .description("Set the timeout (in milliseconds) for unacked messages, truncated to the " - + "nearest millisecond. The timeout needs to be greater than 10 seconds.") + + "nearest millisecond. The timeout needs to be greater than 10 seconds.") .required(false) .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) .defaultValue("10000") .build(); - - public static final PropertyDescriptor PRIORITY_LEVEL = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor PRIORITY_LEVEL = new PropertyDescriptor.Builder() .name("Consumer Priority Level") .description("Sets priority level for the shared subscription consumers to which broker " - + "gives more priority while dispatching messages. Here, broker follows descending " - + "priorities. (eg: 0=max-priority, 1, 2,..) ") + + "gives more priority while dispatching messages. Here, broker follows descending " + + "priorities. (eg: 0=max-priority, 1, 2,..) ") .required(false) .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .defaultValue("5") .build(); - - public static final PropertyDescriptor RECEIVER_QUEUE_SIZE = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor RECEIVER_QUEUE_SIZE = new PropertyDescriptor.Builder() .name("Consumer receive queue size.") .description("The consumer receive queue controls how many messages can be accumulated " - + "by the Consumer before the application calls Consumer.receive(). Using a higher " - + "value could potentially increase the consumer throughput at the expense of bigger " - + "memory utilization. \n" - + "Setting the consumer queue size as zero, \n" - + "\t - Decreases the throughput of the consumer, by disabling pre-fetching of messages. \n" - + "\t - Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer " - + "connection with broker and consumer will not be able receive any further message unless batch-message " - + "in pipeline is removed") + + "by the Consumer before the application calls Consumer.receive(). Using a higher " + + "value could potentially increase the consumer throughput at the expense of bigger " + + "memory utilization. \n" + + "Setting the consumer queue size as zero, \n" + + "\t - Decreases the throughput of the consumer, by disabling pre-fetching of messages. \n" + + "\t - Doesn't support Batch-Message: if consumer receives any batch-message then it closes consumer " + + "connection with broker and consumer will not be able receive any further message unless batch-message " + + "in pipeline is removed") .required(false) .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .defaultValue("1000") .build(); - - public static final PropertyDescriptor SUBSCRIPTION_TYPE = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor SUBSCRIPTION_TYPE = new PropertyDescriptor.Builder() .name("Subscription Type") .description("Select the subscription type to be used when subscribing to the topic.") .required(false) @@ -143,17 +143,17 @@ public class ConsumePulsar_1_0 extends AbstractPulsarProcessor { .defaultValue(SHARED.getValue()) .build(); - private static final List PROPERTIES; + private static final List PROPERTIES; private static final Set RELATIONSHIPS; - + // Reuse the same consumer for a given topic / subscription private PulsarConsumer consumer; private ConsumerConfiguration consumerConfig; - + // Pool for running multiple consume Async requests ExecutorService pool; ExecutorCompletionService completionService; - + static { final List properties = new ArrayList<>(); properties.add(PULSAR_CLIENT_SERVICE); @@ -172,7 +172,7 @@ public class ConsumePulsar_1_0 extends AbstractPulsarProcessor { relationships.add(REL_SUCCESS); RELATIONSHIPS = Collections.unmodifiableSet(relationships); } - + @Override public Set getRelationships() { return RELATIONSHIPS; @@ -182,210 +182,210 @@ public Set getRelationships() { protected List getSupportedPropertyDescriptors() { return PROPERTIES; } - + @OnScheduled public void init(ProcessContext context) { - pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); - completionService = new ExecutorCompletionService<>(pool); + pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); + completionService = new ExecutorCompletionService<>(pool); } - + @OnUnscheduled public void shutDown() { - // Stop all the async consumers - pool.shutdownNow(); + // Stop all the async consumers + pool.shutdownNow(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + try { + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + // Launch consumers + consumeAsync(context, session); + + // Handle completed consumers + handleAsync(context, session); + + } else { + consume(context, session); + } + } catch (PulsarClientException e) { + getLogger().error("Unable to consume from Pulsar Topic ", e); + context.yield(); + throw new ProcessException(e); + } + } - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - - try { - if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { - // Launch consumers - consumeAsync(context, session); - - // Handle completed consumers - handleAsync(context, session); - - } else { - consume(context, session); - } - } catch (PulsarClientException e) { - getLogger().error("Unable to consume from Pulsar Topic ", e); - context.yield(); - throw new ProcessException(e); - } - - } - - private void handleAsync(ProcessContext context, ProcessSession session) { - - try { - Future done = completionService.take(); - Message msg = done.get(); - - if (msg != null) { - FlowFile flowFile = null; - final byte[] value = msg.getData(); - if (value != null && value.length > 0) { - flowFile = session.create(); - flowFile = session.write(flowFile, out -> { - out.write(value); - }); - } - - session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); - session.transfer(flowFile, REL_SUCCESS); - session.commit(); - getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); - } - - } catch (InterruptedException | ExecutionException | PulsarClientException e) { - getLogger().error("Trouble consuming messages ", e); - } - - } - - @OnStopped - public void close(final ProcessContext context) { - - getLogger().info("Disconnecting Pulsar Consumer"); - if (consumer != null) { - - context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class) - .getConsumerPool().evict(consumer); - } - - consumer = null; + + private void handleAsync(ProcessContext context, ProcessSession session) { + + try { + Future done = completionService.take(); + Message msg = done.get(); + + if (msg != null) { + FlowFile flowFile = null; + final byte[] value = msg.getData(); + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + } + + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.transfer(flowFile, REL_SUCCESS); + session.commit(); + getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); + } + + } catch (InterruptedException | ExecutionException | PulsarClientException e) { + getLogger().error("Trouble consuming messages ", e); + } + + } + + @OnStopped + public void close(final ProcessContext context) { + + getLogger().info("Disconnecting Pulsar Consumer"); + if (consumer != null) { + + context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getConsumerPool().evict(consumer); + } + + consumer = null; } - - /* - * For now let's assume that this processor will be configured to run for a longer - * duration than 0 milliseconds. So we will be grabbing as many messages off the topic - * as possible and committing them as FlowFiles - */ + + /* + * For now let's assume that this processor will be configured to run for a longer + * duration than 0 milliseconds. So we will be grabbing as many messages off the topic + * as possible and committing them as FlowFiles + */ private void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException { - - Consumer consumer = getWrappedConsumer(context).getConsumer(); - - completionService.submit(new Callable() { - @Override - public Message call() throws Exception { - return consumer.receiveAsync().get(); - } - }); - + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + completionService.submit(new Callable() { + @Override + public Message call() throws Exception { + return consumer.receiveAsync().get(); + } + }); + } - - /* - * When this Processor expects to receive many small files, it may - * be advisable to create several FlowFiles from a single session - * before committing the session. Typically, this allows the Framework - * to treat the content of the newly created FlowFiles much more efficiently. - */ - private void consume(ProcessContext context, ProcessSession session) throws PulsarClientException { - - Consumer consumer = getWrappedConsumer(context).getConsumer(); - - final ComponentLog logger = getLogger(); - final Message msg; - FlowFile flowFile = null; - + + /* + * When this Processor expects to receive many small files, it may + * be advisable to create several FlowFiles from a single session + * before committing the session. Typically, this allows the Framework + * to treat the content of the newly created FlowFiles much more efficiently. + */ + private void consume(ProcessContext context, ProcessSession session) throws PulsarClientException { + + Consumer consumer = getWrappedConsumer(context).getConsumer(); + + final ComponentLog logger = getLogger(); + final Message msg; + FlowFile flowFile = null; + try { - - msg = consumer.receive(); - final byte[] value = msg.getData(); - if (value != null && value.length > 0) { - flowFile = session.create(); - flowFile = session.write(flowFile, out -> { - out.write(value); - }); + msg = consumer.receive(); + final byte[] value = msg.getData(); + + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); - session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); session.transfer(flowFile, REL_SUCCESS); logger.info("Created {} from {} messages received from Pulsar Server and transferred to 'success'", new Object[]{flowFile, 1}); - + session.commit(); - + /* - * This Processor acknowledges receipt of the data and/or removes the data - * from the external source in order to prevent receipt of duplicate files. - * This is done only after the ProcessSession by which the FlowFile was created - * has been committed! Failure to adhere to this principle may result in data - * loss, as restarting NiFi before the session has been committed will result - * in the temporary file being deleted. Note, however, that it is possible using - * this approach to receive duplicate data because the application could be - * restarted after committing the session and before acknowledging or removing - * the data from the external source. In general, though, potential data duplication + * This Processor acknowledges receipt of the data and/or removes the data + * from the etXernal source in order to prevent receipt of duplicate files. + * This is done only after the ProcessSession by which the FlowFile was created + * has been committed! Failure to adhere to this principle may result in data + * loss, as restarting NiFi before the session has been committed will result + * in the temporary file being deleted. Note, however, that it is possible using + * this approach to receive duplicate data because the application could be + * restarted after committing the session and before acknowledging or removing + * the data from the etXernal source. In general, though, potential data duplication * is preferred over potential data loss. */ getLogger().info("Acknowledging message " + msg.getMessageId()); - consumer.acknowledge(msg); - - } else { - // We didn't consume any data, so - session.commit(); - } - - } catch (PulsarClientException e) { - context.yield(); - session.rollback(); - } - - } - - private PulsarConsumer getWrappedConsumer(ProcessContext context) throws PulsarClientException { - - if (consumer != null) - return consumer; - - final PulsarClientPool pulsarClientService = context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class); - - try { - consumer = pulsarClientService.getConsumerPool() - .acquire(getConsumerProperties(context)); - - if (consumer == null || consumer.getConsumer() == null) { - throw new PulsarClientException("Unable to create Pulsar Consumer"); - } - - return consumer; - } catch (final InterruptedException ex) { - return null; - } - } - - private Properties getConsumerProperties(ProcessContext context) { - - Properties props = new Properties(); - props.put(PulsarConsumerFactory.TOPIC_NAME, context.getProperty(TOPIC).getValue()); - props.put(PulsarConsumerFactory.SUBSCRIPTION_NAME, context.getProperty(SUBSCRIPTION).getValue()); - props.put(PulsarConsumerFactory.CONSUMER_CONFIG, getConsumerConfig(context)); - return props; - } - + consumer.acknowledge(msg); + + } else { + // We didn't consume any data, so + session.commit(); + } + + } catch (PulsarClientException e) { + context.yield(); + session.rollback(); + } + + } + + private PulsarConsumer getWrappedConsumer(ProcessContext context) throws PulsarClientException { + + if (consumer != null) + return consumer; + + final PulsarClientPool pulsarClientService = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class); + + try { + consumer = pulsarClientService.getConsumerPool() + .acquire(getConsumerProperties(context)); + + if (consumer == null || consumer.getConsumer() == null) { + throw new PulsarClientException("Unable to create Pulsar Consumer"); + } + + return consumer; + } catch (final InterruptedException ex) { + return null; + } + } + + private Properties getConsumerProperties(ProcessContext context) { + + Properties props = new Properties(); + props.put(PulsarConsumerFactory.TOPIC_NAME, context.getProperty(TOPIC).getValue()); + props.put(PulsarConsumerFactory.SUBSCRIPTION_NAME, context.getProperty(SUBSCRIPTION).getValue()); + props.put(PulsarConsumerFactory.CONSUMER_CONFIG, getConsumerConfig(context)); + return props; + } + private ConsumerConfiguration getConsumerConfig(ProcessContext context) { - - if (consumerConfig == null) { - consumerConfig = new ConsumerConfiguration(); - - if (context.getProperty(ACK_TIMEOUT).isSet()) - consumerConfig.setAckTimeout(context.getProperty(ACK_TIMEOUT).asLong(), TimeUnit.MILLISECONDS); - - if (context.getProperty(PRIORITY_LEVEL).isSet()) - consumerConfig.setPriorityLevel(context.getProperty(PRIORITY_LEVEL).asInteger()); - - if (context.getProperty(RECEIVER_QUEUE_SIZE).isSet()) - consumerConfig.setReceiverQueueSize(context.getProperty(RECEIVER_QUEUE_SIZE).asInteger()); - - if (context.getProperty(SUBSCRIPTION_TYPE).isSet()) - consumerConfig.setSubscriptionType(SubscriptionType.valueOf(context.getProperty(SUBSCRIPTION_TYPE).getValue())); - } - - return consumerConfig; + + if (consumerConfig == null) { + consumerConfig = new ConsumerConfiguration(); + + if (context.getProperty(ACK_TIMEOUT).isSet()) + consumerConfig.setAckTimeout(context.getProperty(ACK_TIMEOUT).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(PRIORITY_LEVEL).isSet()) + consumerConfig.setPriorityLevel(context.getProperty(PRIORITY_LEVEL).asInteger()); + + if (context.getProperty(RECEIVER_QUEUE_SIZE).isSet()) + consumerConfig.setReceiverQueueSize(context.getProperty(RECEIVER_QUEUE_SIZE).asInteger()); + + if (context.getProperty(SUBSCRIPTION_TYPE).isSet()) + consumerConfig.setSubscriptionType(SubscriptionType.valueOf(context.getProperty(SUBSCRIPTION_TYPE).getValue())); + } + + return consumerConfig; } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java index 028d7061d00d..52b737b99a56 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java @@ -62,119 +62,119 @@ + "The complementary NiFi processor for fetching messages is ConsumePulsar.") @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " - + "FlowFiles that are routed to success.") + + "FlowFiles that are routed to success.") public class PublishPulsar_1_0 extends AbstractPulsarProcessor { - - protected static final String MSG_COUNT = "msg.count"; - - static final AllowableValue COMPRESSION_TYPE_NONE = new AllowableValue("NONE", "None", "No compression"); - static final AllowableValue COMPRESSION_TYPE_LZ4 = new AllowableValue("LZ4", "LZ4", "Compress with LZ4 algorithm."); - static final AllowableValue COMPRESSION_TYPE_ZLIB = new AllowableValue("ZLIB", "ZLIB", "Compress with ZLib algorithm"); - - static final AllowableValue MESSAGE_ROUTING_MODE_CUSTOM_PARTITION = new AllowableValue("CustomPartition", "Custom Partition", "Route messages to a custom partition"); - static final AllowableValue MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION = new AllowableValue("RoundRobinPartition", "Round Robin Partition", "Route messages to all partitions in a round robin manner"); - static final AllowableValue MESSAGE_ROUTING_MODE_SINGLE_PARTITION = new AllowableValue("SinglePartition", "Single Partition", "Route messages to a single partition"); - - public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() - .name("topic") - .displayName("Topic Name") - .description("The name of the Pulsar Topic.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .expressionLanguageSupported(true) - .build(); - - public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() + + protected static final String MSG_COUNT = "msg.count"; + + static final AllowableValue COMPRESSION_TYPE_NONE = new AllowableValue("NONE", "None", "No compression"); + static final AllowableValue COMPRESSION_TYPE_LZ4 = new AllowableValue("LZ4", "LZ4", "Compress with LZ4 algorithm."); + static final AllowableValue COMPRESSION_TYPE_ZLIB = new AllowableValue("ZLIB", "ZLIB", "Compress with ZLib algorithm"); + static final AllowableValue MESSAGE_ROUTING_MODE_CUSTOM_PARTITION = new AllowableValue("CustomPartition", "Custom Partition", "Route messages to a custom partition"); + static final AllowableValue MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION = new AllowableValue("RoundRobinPartition", "Round Robin Partition", "Route messages to all partitions " + + "in a round robin manner"); + static final AllowableValue MESSAGE_ROUTING_MODE_SINGLE_PARTITION = new AllowableValue("SinglePartition", "Single Partition", "Route messages to a single partition"); + + public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Pulsar Topic.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor ASYNC_ENABLED = new PropertyDescriptor.Builder() .name("Async Enabled") .description("Control whether the messages will be sent asyncronously or not. Messages sent" - + " syncronously will be acknowledged immediately before processing the next message, while" - + " asyncronous messages will be acknowledged after the Pulsar broker responds.") + + " syncronously will be acknowledged immediately before processing the netX message, while" + + " asyncronous messages will be acknowledged after the Pulsar broker responds.") .required(true) .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); - - public static final PropertyDescriptor BATCHING_ENABLED = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor BATCHING_ENABLED = new PropertyDescriptor.Builder() .name("Batching Enabled") .description("Control whether automatic batching of messages is enabled for the producer. " - + "default: false [No batching] When batching is enabled, multiple calls to " - + "Producer.sendAsync can result in a single batch to be sent to the broker, leading " - + "to better throughput, especially when publishing small messages. If compression is " - + "enabled, messages will be compressed at the batch level, leading to a much better " - + "compression ratio for similar headers or contents. When enabled default batch delay " - + "is set to 10 ms and default batch size is 1000 messages") + + "default: false [No batching] When batching is enabled, multiple calls to " + + "Producer.sendAsync can result in a single batch to be sent to the broker, leading " + + "to better throughput, especially when publishing small messages. If compression is " + + "enabled, messages will be compressed at the batch level, leading to a much better " + + "compression ratio for similar headers or contents. When enabled default batch delay " + + "is set to 10 ms and default batch size is 1000 messages") .required(false) .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); - - public static final PropertyDescriptor BATCHING_MAX_MESSAGES = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor BATCHING_MAX_MESSAGES = new PropertyDescriptor.Builder() .name("Batching Max Messages") .description("Set the maximum number of messages permitted in a batch. default: " - + "1000 If set to a value greater than 1, messages will be queued until this " - + "threshold is reached or batch interval has elapsed") + + "1000 If set to a value greater than 1, messages will be queued until this " + + "threshold is reached or batch interval has elapsed") .required(false) .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .defaultValue("1000") .build(); - - public static final PropertyDescriptor BATCH_INTERVAL = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor BATCH_INTERVAL = new PropertyDescriptor.Builder() .name("Batch Interval") .description("Set the time period within which the messages sent will be batched default: 10ms " - + "if batch messages are enabled. If set to a non zero value, messages will be queued until " - + "this time interval or until the Batching Max Messages threshould has been reached") + + "if batch messages are enabled. If set to a non zero value, messages will be queued until " + + "this time interval or until the Batching Max Messages threshould has been reached") .required(false) .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) .defaultValue("10") .build(); - - public static final PropertyDescriptor BLOCK_IF_QUEUE_FULL = new PropertyDescriptor.Builder() - .name("Block if Message Queue Full") - .description("Set whether the processor should block when the outgoing message queue is full. " - + "Default is false. If set to false, send operations will immediately fail with " - + "ProducerQueueIsFullError when there is no space left in pending queue.") - .required(false) + + public static final PropertyDescriptor BLOCK_IF_QUEUE_FULL = new PropertyDescriptor.Builder() + .name("Block if Message Queue Full") + .description("Set whether the processor should block when the outgoing message queue is full. " + + "Default is false. If set to false, send operations will immediately fail with " + + "ProducerQueueIsFullError when there is no space left in pending queue.") + .required(false) .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); - - public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder() .name("Compression Type") .description("Set the compression type for the producer.") .required(false) .allowableValues(COMPRESSION_TYPE_NONE, COMPRESSION_TYPE_LZ4, COMPRESSION_TYPE_ZLIB) .defaultValue(COMPRESSION_TYPE_NONE.getValue()) .build(); - - public static final PropertyDescriptor MESSAGE_ROUTING_MODE = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor MESSAGE_ROUTING_MODE = new PropertyDescriptor.Builder() .name("Message Routing Mode") .description("Set the message routing mode for the producer. This applies only if the destination topic is partitioned") .required(false) .allowableValues(MESSAGE_ROUTING_MODE_CUSTOM_PARTITION, MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION, MESSAGE_ROUTING_MODE_SINGLE_PARTITION) .defaultValue(MESSAGE_ROUTING_MODE_ROUND_ROBIN_PARTITION.getValue()) .build(); - - public static final PropertyDescriptor PENDING_MAX_MESSAGES = new PropertyDescriptor.Builder() + + public static final PropertyDescriptor PENDING_MAX_MESSAGES = new PropertyDescriptor.Builder() .name("Max Pending Messages") .description("Set the max size of the queue holding the messages pending to receive an " - + "acknowledgment from the broker.") + + "acknowledgment from the broker.") .required(false) .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .defaultValue("1000") .build(); - - private static final List PROPERTIES; + + private static final List PROPERTIES; private static final Set RELATIONSHIPS; - + private LRUCache producers; private ProducerConfiguration producerConfig; - + static { final List properties = new ArrayList<>(); properties.add(PULSAR_CLIENT_SERVICE); properties.add(TOPIC); properties.add(ASYNC_ENABLED); - properties.add(BATCHING_ENABLED); + properties.add(BATCHING_ENABLED); properties.add(BATCHING_MAX_MESSAGES); properties.add(BATCH_INTERVAL); properties.add(BLOCK_IF_QUEUE_FULL); @@ -189,7 +189,7 @@ public class PublishPulsar_1_0 extends AbstractPulsarProcessor { relationships.add(REL_FAILURE); RELATIONSHIPS = Collections.unmodifiableSet(relationships); } - + @Override public Set getRelationships() { return RELATIONSHIPS; @@ -199,174 +199,173 @@ public Set getRelationships() { protected List getSupportedPropertyDescriptors() { return PROPERTIES; } - + @OnStopped public void cleanUp(final ProcessContext context) { - // Close all of the producers and invalidate them, so they get removed from the Resource Pool - getProducerCache(context).clear(); + // Close all of the producers and invalidate them, so they get removed from the Resource Pool + getProducerCache(context).clear(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + FlowFile flowFile = session.get(); + + if (flowFile == null) + return; + + final ComponentLog logger = getLogger(); + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + + if (StringUtils.isBlank(topic)) { + logger.error("Invalid topic specified {}", new Object[] {topic}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + // Read the contents of the FlowFile into a byte array + final byte[] messageContent = new byte[(int) flowFile.getSize()]; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + StreamUtils.fillBuffer(in, messageContent, true); + } + }); + + // Nothing to do, so skip this Flow file. + if (messageContent == null || messageContent.length < 1) { + session.transfer(flowFile, REL_SUCCESS); + return; + } + + try { + + Producer producer = getWrappedProducer(topic, context).getProducer(); + + if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + this.sendAsync(producer, session, flowFile, messageContent); + } else { + this.send(producer, session, flowFile, messageContent); + } + + } catch (final PulsarClientException e) { + logger.error("Failed to connect to Pulsar Server due to {}", new Object[]{e}); + session.penalize(flowFile); + session.transfer(flowFile, REL_FAILURE); + } + } - - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - - FlowFile flowFile = session.get(); - - if (flowFile == null) - return; - - final ComponentLog logger = getLogger(); - final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); - - if (StringUtils.isBlank(topic)) { - logger.error("Invalid topic specified {}", new Object[] {topic}); - session.transfer(flowFile, REL_FAILURE); - return; - } - - // Read the contents of the FlowFile into a byte array - final byte[] messageContent = new byte[(int) flowFile.getSize()]; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - StreamUtils.fillBuffer(in, messageContent, true); - } - }); - - // Nothing to do, so skip this Flow file. - if (messageContent == null || messageContent.length < 1) { - session.transfer(flowFile, REL_SUCCESS); - return; - } - - try { - - Producer producer = getWrappedProducer(topic, context).getProducer(); - - if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { - this.sendAsync(producer, session, flowFile, messageContent); - } else { - this.send(producer, session, flowFile, messageContent); - } - - } catch (final PulsarClientException e) { - logger.error("Failed to connect to Pulsar Server due to {}", new Object[]{e}); - session.penalize(flowFile); - session.transfer(flowFile, REL_FAILURE); - } - - } - + private void send(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) throws PulsarClientException { - - MessageId msgId = producer.send(messageContent); - - if (msgId != null) { - - flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); - session.adjustCounter("Messages Sent", 1, true); - session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); - session.transfer(flowFile, REL_SUCCESS); - - } else { - session.transfer(flowFile, REL_FAILURE); - } - + + MessageId msgId = producer.send(messageContent); + + if (msgId != null) { + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } else { + session.transfer(flowFile, REL_FAILURE); + } + } - + private void sendAsync(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) { - - producer.sendAsync(messageContent).handle((msgId, ex) -> { - if (msgId != null) { - return msgId; - } else { - // TODO Communicate the error back up to the onTrigger method so we can invalidate this producer. - getLogger().warn("Problem ", ex); - return null; - } - }); - - flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); - session.adjustCounter("Messages Sent", 1, true); - session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); - session.transfer(flowFile, REL_SUCCESS); - + + producer.sendAsync(messageContent).handle((msgId, ex) -> { + if (msgId != null) { + return msgId; + } else { + // TODO Communicate the error back up to the onTrigger method so we can invalidate this producer. + getLogger().warn("Problem ", ex); + return null; + } + }); + + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); + + } + + private PulsarProducer getWrappedProducer(String topic, ProcessContext context) throws PulsarClientException, IllegalArgumentException { + + PulsarProducer producer = getProducerCache(context).get(topic); + + if (producer != null) + return producer; + + try { + producer = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool().acquire(getProducerProperties(context, topic)); + + if (producer != null) { + producers.put(topic, producer); + } + + return producer; + + } catch (InterruptedException e) { + return null; + } + } - - private PulsarProducer getWrappedProducer(String topic, ProcessContext context) throws PulsarClientException, IllegalArgumentException { - - PulsarProducer producer = getProducerCache(context).get(topic); - - if (producer != null) - return producer; - - try { - producer = context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class) - .getProducerPool().acquire(getProducerProperties(context, topic)); - - if (producer != null) { - producers.put(topic, producer); - } - - return producer; - - } catch (InterruptedException e) { - return null; - } - - } - - private LRUCache getProducerCache(ProcessContext context) { - if (producers == null) { - - ResourcePool pool = context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class) - .getProducerPool(); - - producers = new LRUCache (20, pool); - } - - return producers; - } - - private Properties getProducerProperties(ProcessContext context, String topic) { - - Properties props = new Properties(); - props.put(PulsarProducerFactory.TOPIC_NAME, topic); - props.put(PulsarProducerFactory.PRODUCER_CONFIG, getProducerConfig(context)); - return props; - } - + + private LRUCache getProducerCache(ProcessContext context) { + if (producers == null) { + + ResourcePool pool = context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getProducerPool(); + + producers = new LRUCache(20, pool); + } + + return producers; + } + + private Properties getProducerProperties(ProcessContext context, String topic) { + + Properties props = new Properties(); + props.put(PulsarProducerFactory.TOPIC_NAME, topic); + props.put(PulsarProducerFactory.PRODUCER_CONFIG, getProducerConfig(context)); + return props; + } + private ProducerConfiguration getProducerConfig(ProcessContext context) { - - if (producerConfig == null) { - producerConfig = new ProducerConfiguration(); - - if (context.getProperty(BATCHING_ENABLED).isSet()) - producerConfig.setBatchingEnabled(context.getProperty(BATCHING_ENABLED).asBoolean()); - - if (context.getProperty(BATCHING_MAX_MESSAGES).isSet()) - producerConfig.setBatchingMaxMessages(context.getProperty(BATCHING_MAX_MESSAGES).asInteger()); - - if (context.getProperty(BATCH_INTERVAL).isSet()) - producerConfig.setBatchingMaxPublishDelay(context.getProperty(BATCH_INTERVAL).asLong(), TimeUnit.MILLISECONDS); - - if (context.getProperty(BLOCK_IF_QUEUE_FULL).isSet()) - producerConfig.setBlockIfQueueFull(context.getProperty(BLOCK_IF_QUEUE_FULL).asBoolean()); - - if (context.getProperty(COMPRESSION_TYPE).isSet()) - producerConfig.setCompressionType(CompressionType.valueOf(context.getProperty(COMPRESSION_TYPE).getValue())); - - if (context.getProperty(PENDING_MAX_MESSAGES).isSet()) - producerConfig.setMaxPendingMessages(context.getProperty(PENDING_MAX_MESSAGES).asInteger()); - - if (context.getProperty(MESSAGE_ROUTING_MODE).isSet()) - producerConfig.setMessageRoutingMode(MessageRoutingMode.valueOf(context.getProperty(MESSAGE_ROUTING_MODE).getValue())); - } - - return producerConfig; + + if (producerConfig == null) { + producerConfig = new ProducerConfiguration(); + + if (context.getProperty(BATCHING_ENABLED).isSet()) + producerConfig.setBatchingEnabled(context.getProperty(BATCHING_ENABLED).asBoolean()); + + if (context.getProperty(BATCHING_MAX_MESSAGES).isSet()) + producerConfig.setBatchingMaxMessages(context.getProperty(BATCHING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(BATCH_INTERVAL).isSet()) + producerConfig.setBatchingMaxPublishDelay(context.getProperty(BATCH_INTERVAL).asLong(), TimeUnit.MILLISECONDS); + + if (context.getProperty(BLOCK_IF_QUEUE_FULL).isSet()) + producerConfig.setBlockIfQueueFull(context.getProperty(BLOCK_IF_QUEUE_FULL).asBoolean()); + + if (context.getProperty(COMPRESSION_TYPE).isSet()) + producerConfig.setCompressionType(CompressionType.valueOf(context.getProperty(COMPRESSION_TYPE).getValue())); + + if (context.getProperty(PENDING_MAX_MESSAGES).isSet()) + producerConfig.setMaxPendingMessages(context.getProperty(PENDING_MAX_MESSAGES).asInteger()); + + if (context.getProperty(MESSAGE_ROUTING_MODE).isSet()) + producerConfig.setMessageRoutingMode(MessageRoutingMode.valueOf(context.getProperty(MESSAGE_ROUTING_MODE).getValue())); + } + + return producerConfig; } - + } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java index 1bee9dc120a5..9110aaf85452 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -22,13 +22,13 @@ import org.mockito.Mock; public abstract class AbstractPulsarProcessorTest { - - protected TestRunner runner; - + + protected TestRunner runner; + @Mock - protected PulsarClient mockClient; + protected PulsarClient mockClient; - protected void addPulsarClientService() throws InitializationException { + protected void addPulsarClientService() throws InitializationException { final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); runner.addControllerService("pulsarClient", pulsarClient); runner.enableControllerService(pulsarClient); diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java index 6f138c0442c7..5175b93d8246 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -32,7 +32,10 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -40,172 +43,121 @@ public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { @Mock - Consumer mockConsumer; - + Consumer mockConsumer; + @Mock - Message mockMessage; - + Message mockMessage; + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); @Before public void init() throws InitializationException { runner = TestRunners.newTestRunner(ConsumePulsar_1_0.class); - mockClient = mock(PulsarClient.class); mockConsumer = mock(Consumer.class); mockMessage = mock(Message.class); - + try { - when(mockClient.subscribe(anyString(), anyString())).thenReturn(mockConsumer); - when(mockConsumer.receive()).thenReturn(mockMessage); - - CompletableFuture future = CompletableFuture.supplyAsync(() -> { - return mockMessage; - }); - - when(mockConsumer.receiveAsync()).thenReturn(future); - - } catch (PulsarClientException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - + when(mockClient.subscribe(anyString(), anyString())).thenReturn(mockConsumer); + when(mockConsumer.receive()).thenReturn(mockMessage); + + CompletableFuture future = CompletableFuture.supplyAsync(() -> { + return mockMessage; + }); + + when(mockConsumer.receiveAsync()).thenReturn(future); + + } catch (PulsarClientException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + addPulsarClientService(); } - + @Test public void emptyMessageTest() { - when(mockMessage.getData()).thenReturn("".getBytes()); - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); - runner.run(); - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); -======= - runner.setProperty(ConsumePulsar.TOPIC, "foo"); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); - runner.run(); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); - runner.run(); - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 + when(mockMessage.getData()).thenReturn("".getBytes()); + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); + runner.run(); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); } - + @Test - public void singleSyncMessageTest() throws PulsarClientException { - this.sendMessages("Mocked Message", "foo", "bar", false, 1); + public void singleSyncMessageTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", false, 1); } - + @Test public void multipleSyncMessagesTest() throws PulsarClientException { - this.sendMessages("Mocked Message", "foo", "bar", false, 40); + this.sendMessages("Mocked Message", "foo", "bar", false, 40); } - + @Test - public void singleAsyncMessageTest() throws PulsarClientException { - this.sendMessages("Mocked Message", "foo", "bar", true, 1); + public void singleAsyncMessageTest() throws PulsarClientException { + this.sendMessages("Mocked Message", "foo", "bar", true, 1); } - + @Test public void multipleAsyncMessagesTest() throws PulsarClientException { - this.sendMessages("Mocked Message", "foo", "bar", true, 40); + this.sendMessages("Mocked Message", "foo", "bar", true, 40); } - + /* - * Verify that the consumer gets closed. + * Verify that the consumer gets closed. */ @Test public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { - when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); - runner.run(10, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); -======= - runner.setProperty(ConsumePulsar.TOPIC, "foo"); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, "bar"); - runner.run(10, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar.REL_SUCCESS); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); - runner.run(10, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - - runner.assertQueueEmpty(); - + when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); + + runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); + runner.run(10, true); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); + runner.assertQueueEmpty(); + // Verify that the receive method on the consumer was called 10 times verify(mockConsumer, times(10)).receive(); - + // Verify that each message was acknowledged verify(mockConsumer, times(10)).acknowledge(mockMessage); - + // Verify that the consumer was closed verify(mockConsumer, times(1)).close(); - + } private void sendMessages(String msg, String topic, String sub, boolean async, int itertions) throws PulsarClientException { - - when(mockMessage.getData()).thenReturn(msg.getBytes()); - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); - runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); - runner.run(itertions, true); - - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); - - List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_0.REL_SUCCESS); -======= - runner.setProperty(ConsumePulsar.ASYNC_ENABLED, Boolean.toString(async)); - runner.setProperty(ConsumePulsar.TOPIC, topic); - runner.setProperty(ConsumePulsar.SUBSCRIPTION, sub); -======= - runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); - runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - runner.run(itertions, true); - - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); - -<<<<<<< HEAD - List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar.REL_SUCCESS); ->>>>>>> Added Pulsar processors and Controller Service -======= + + when(mockMessage.getData()).thenReturn(msg.getBytes()); + + runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); + runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); + runner.run(itertions, true); + + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_0.REL_SUCCESS); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 + assertEquals(itertions, flowFiles.size()); - + for (MockFlowFile ff : flowFiles) { - ff.assertContentEquals(msg); + ff.assertContentEquals(msg); } - + if (async) { - verify(mockConsumer, times(itertions)).receiveAsync(); + verify(mockConsumer, times(itertions)).receiveAsync(); } else { - verify(mockConsumer, times(itertions)).receive(); + verify(mockConsumer, times(itertions)).receive(); } - + // Verify that every message was acknowledged if (async) { - verify(mockConsumer, times(itertions)).acknowledgeAsync(mockMessage); + verify(mockConsumer, times(itertions)).acknowledgeAsync(mockMessage); } else { - verify(mockConsumer, times(itertions)).acknowledge(mockMessage); + verify(mockConsumer, times(itertions)).acknowledge(mockMessage); } } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java index 3f7829272eb0..d622b57e1859 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java @@ -36,121 +36,124 @@ import org.mockito.junit.MockitoRule; import org.mockito.stubbing.Answer; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.util.Properties; public class MockPulsarClientService extends AbstractControllerService implements PulsarClientPool { - @Mock - PulsarClient mockClient; - - @Mock - ResourcePool mockProducerPool; - - @Mock - ResourcePool mockConsumerPool; - - @Mock - Producer mockProducer; - - @Mock - Consumer mockConsumer; - - @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); - - @SuppressWarnings({ "unchecked", "rawtypes" }) - public MockPulsarClientService(PulsarClient mockClient2) { - this.mockClient = mockClient2; - mockProducerPool = mock(ResourcePool.class); - mockConsumerPool = mock(ResourcePool.class); - mockProducer = mock(Producer.class); - mockConsumer = mock(Consumer.class); - - try { - when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( - new Answer () { - @Override - public PulsarProducer answer(InvocationOnMock invocation) { - Properties props = invocation.getArgumentAt(0, Properties.class); - return getProducer(props); - } - } - ); - - when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( - new Answer () { - @Override - public PulsarConsumer answer(InvocationOnMock invocation) { - Properties props = invocation.getArgumentAt(0, Properties.class); - return getConsumer(props); - } - } - ); - - - doAnswer(new Answer() { - public Object answer(InvocationOnMock invocation){ - PulsarConsumer consumer = invocation.getArgumentAt(0, PulsarConsumer.class); - consumer.close(); - return null; - } - }).when(mockConsumerPool).evict(any(PulsarConsumer.class)); - - } catch (InterruptedException ex) { - - } - - try { - when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { - @Override - public boolean matches(Object argument) { - return true; - } - }))).thenReturn(null); - } catch (PulsarClientException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - - public Producer getMockProducer() { - return mockProducer; - } - - public PulsarClient getMockClient() { - return mockClient; - } - - public PulsarProducer getProducer(Properties props) { - String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); - try { - return new PulsarProducer(mockClient.createProducer(topic), topic); - } catch (PulsarClientException e) { - return null; - } - } - - public PulsarConsumer getConsumer(Properties props) { - String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); - String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); - try { - return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); - } catch (PulsarClientException e) { - return null; - } - } - - @Override - public ResourcePool getProducerPool() { - return mockProducerPool; - } - - - @Override - public ResourcePool getConsumerPool() { - return mockConsumerPool; - } - + @Mock + PulsarClient mockClient; + + @Mock + ResourcePool mockProducerPool; + + @Mock + ResourcePool mockConsumerPool; + + @Mock + Producer mockProducer; + + @Mock + Consumer mockConsumer; + + @Rule public MockitoRule mockitoRule = MockitoJUnit.rule(); + + @SuppressWarnings({ "unchecked", "rawtypes" }) + public MockPulsarClientService(PulsarClient mockClient2) { + this.mockClient = mockClient2; + mockProducerPool = mock(ResourcePool.class); + mockConsumerPool = mock(ResourcePool.class); + mockProducer = mock(Producer.class); + mockConsumer = mock(Consumer.class); + + try { + when(mockProducerPool.acquire(any(Properties.class))).thenAnswer( + new Answer() { + @Override + public PulsarProducer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getProducer(props); + } + } + ); + + when(mockConsumerPool.acquire(any(Properties.class))).thenAnswer( + new Answer() { + @Override + public PulsarConsumer answer(InvocationOnMock invocation) { + Properties props = invocation.getArgumentAt(0, Properties.class); + return getConsumer(props); + } + } + ); + + + doAnswer(new Answer() { + public Object answer(InvocationOnMock invocation){ + PulsarConsumer consumer = invocation.getArgumentAt(0, PulsarConsumer.class); + consumer.close(); + return null; + } + }).when(mockConsumerPool).evict(any(PulsarConsumer.class)); + + } catch (InterruptedException ex) { + + } + + try { + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(null); + } catch (PulsarClientException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + + + public Producer getMockProducer() { + return mockProducer; + } + + public PulsarClient getMockClient() { + return mockClient; + } + + public PulsarProducer getProducer(Properties props) { + String topic = props.getProperty(PulsarProducerFactory.TOPIC_NAME); + try { + return new PulsarProducer(mockClient.createProducer(topic), topic); + } catch (PulsarClientException e) { + return null; + } + } + + public PulsarConsumer getConsumer(Properties props) { + String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); + String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); + try { + return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + } catch (PulsarClientException e) { + return null; + } + } + + @Override + public ResourcePool getProducerPool() { + return mockProducerPool; + } + + + @Override + public ResourcePool getConsumerPool() { + return mockConsumerPool; + } + } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java index 67b738710a4b..a229e06c3fb8 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -35,12 +35,15 @@ import org.mockito.Mock; import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { - + @Mock - Producer mockProducer; + Producer mockProducer; @Before public void init() throws InitializationException { @@ -49,293 +52,162 @@ public void init() throws InitializationException { mockClient = mock(PulsarClient.class); mockProducer = mock(Producer.class); - + try { - // Use the mockProducer for all Producer interactions - when(mockClient.createProducer(anyString())).thenReturn(mockProducer); - - when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { - @Override - public boolean matches(Object argument) { - return true; - } - }))).thenReturn(mock(MessageId.class)); - - CompletableFuture future = CompletableFuture.supplyAsync(() -> { - return mock(MessageId.class); - }); - - when(mockProducer.sendAsync(Matchers.argThat(new ArgumentMatcher() { - @Override - public boolean matches(Object argument) { - return true; - } - }))).thenReturn(future); - - - } catch (PulsarClientException e) { - e.printStackTrace(); - } - + // Use the mockProducer for all Producer interactions + when(mockClient.createProducer(anyString())).thenReturn(mockProducer); + + when(mockProducer.send(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(mock(MessageId.class)); + + CompletableFuture future = CompletableFuture.supplyAsync(() -> { + return mock(MessageId.class); + }); + + when(mockProducer.sendAsync(Matchers.argThat(new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + return true; + } + }))).thenReturn(future); + + + } catch (PulsarClientException e) { + e.printStackTrace(); + } + addPulsarClientService(); } - - @Test - public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); - - final String content = "some content"; - Map attributes = new HashMap (); - attributes.put(PublishPulsar_1_0.TOPIC.getName(), ""); - - runner.enqueue(content.getBytes("UTF-8"), attributes ); - runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); -======= - runner.setProperty(PublishPulsar.TOPIC, "${topic}"); -======= - runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - - final String content = "some content"; - Map attributes = new HashMap (); + + @Test + public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap(); attributes.put(PublishPulsar_1_0.TOPIC.getName(), ""); - - runner.enqueue(content.getBytes("UTF-8"), attributes ); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); -<<<<<<< HEAD - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_FAILURE); ->>>>>>> Added Pulsar processors and Controller Service -======= runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - - // Confirm that no Producer as created + + // Confirm that no Producer as created verify(mockClient, times(0)).createProducer(anyString()); - } - - @Test - public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); - - final String content = "some content"; - Map attributes = new HashMap (); - attributes.put(PublishPulsar_1_0.TOPIC.getName(), "topic-b"); - - runner.enqueue(content.getBytes("UTF-8"), attributes ); - runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); -======= - runner.setProperty(PublishPulsar.TOPIC, "${topic}"); -======= - runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - - final String content = "some content"; - Map attributes = new HashMap (); + } + + @Test + public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); + + final String content = "some content"; + Map attributes = new HashMap(); attributes.put(PublishPulsar_1_0.TOPIC.getName(), "topic-b"); - - runner.enqueue(content.getBytes("UTF-8"), attributes ); + + runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); -<<<<<<< HEAD - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); ->>>>>>> Added Pulsar processors and Controller Service -======= runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - + // Verify that we sent the data to topic-b. verify(mockClient, times(1)).createProducer("topic-b"); - } + } - @Test + @Test public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); -======= - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - - final String content = "some content"; + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + + final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); -<<<<<<< HEAD -<<<<<<< HEAD - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); - - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); -======= - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); - - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); ->>>>>>> Added Pulsar processors and Controller Service -======= runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 outFile.assertContentEquals(content); - + // Verify that we sent the data to my-topic. verify(mockClient, times(1)).createProducer("my-topic"); - + // Verify that the send method on the producer was called with the expected content verify(mockProducer, times(1)).send(content.getBytes()); - } - - @Test + } + + @Test public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); -======= - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - - final String content = "some content"; + + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); + + final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); -<<<<<<< HEAD -<<<<<<< HEAD runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); -======= - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar.REL_SUCCESS).get(0); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); - - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 outFile.assertContentEquals(content); - + // Verify that we sent the data to my-topic. verify(mockClient, times(1)).createProducer("my-topic"); - + // Verify that the send method on the producer was called with the expected content verify(mockProducer, times(1)).sendAsync(content.getBytes()); - } - - @Test - public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); -======= - runner.setProperty(PublishPulsar.TOPIC, "my-topic"); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - final String content = "some content"; - - // Hack, since runner.run(20, false); doesn't work as advertised - for (int idx = 0; idx < 20; idx++) { - runner.enqueue(content.getBytes("UTF-8")); - runner.run(); -<<<<<<< HEAD -<<<<<<< HEAD - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); -======= - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - } - + } + + @Test + public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + final String content = "some content"; + + // Hack, since runner.run(20, false); doesn't work as advertised + for (int idx = 0; idx < 20; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + + } + // Verify that the send method on the producer was called with the expected content verify(mockProducer, times(20)).send(content.getBytes()); - } - - @Test - public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); -======= - runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - final String content = "some content"; - - // Hack, since runner.run(20, false); doesn't work as advertised - for (int idx = 0; idx < 20; idx++) { - runner.enqueue(content.getBytes("UTF-8")); - runner.run(); -<<<<<<< HEAD -<<<<<<< HEAD - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); -======= - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - } - + } + + @Test + public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { + + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); + + final String content = "some content"; + + // Hack, since runner.run(20, false); doesn't work as advertised + for (int idx = 0; idx < 20; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + } + // Verify that the send method on the producer was called with the expected content verify(mockProducer, times(20)).sendAsync(content.getBytes()); - } - - @Test - public void stressTest() throws UnsupportedEncodingException { -<<<<<<< HEAD -<<<<<<< HEAD - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); -======= - runner.setProperty(PublishPulsar.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar.ASYNC_ENABLED, Boolean.TRUE.toString()); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - final String content = "some content"; - - for (int idx = 0; idx < 9999; idx++) { - runner.enqueue(content.getBytes("UTF-8")); - runner.run(); -<<<<<<< HEAD -<<<<<<< HEAD - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); -======= - runner.assertAllFlowFilesTransferred(PublishPulsar.REL_SUCCESS); ->>>>>>> Added Pulsar processors and Controller Service -======= - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); ->>>>>>> Refactored Processors to indicate the supported version of Pulsar, i.e _1_0 - } - - } + } + + @Test + public void stressTest() throws UnsupportedEncodingException { + runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); + + final String content = "some content"; + + for (int idx = 0; idx < 9999; idx++) { + runner.enqueue(content.getBytes("UTF-8")); + runner.run(); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + } + + } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml index abb745eb259f..5a6275126d17 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml @@ -30,6 +30,10 @@ nifi-pulsar-processors nifi-pulsar-nar + nifi-pulsar-client-service-api + + nifi-pulsar-client-service + diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml index a1d6eacd7447..526f19f436d6 100755 --- a/nifi-nar-bundles/pom.xml +++ b/nifi-nar-bundles/pom.xml @@ -93,6 +93,9 @@ nifi-spark-bundle nifi-atlas-bundle nifi-druid-bundle + nifi-pulsar-bundle + + From 03e8ef2a19381cd3249eb45c471c36b4ae265566 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 9 Mar 2018 22:26:46 -0800 Subject: [PATCH 11/16] Fixed parent.relativePath in POM files --- .../nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml | 2 +- .../nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml | 2 +- nifi-nar-bundles/nifi-pulsar-bundle/pom.xml | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml index 2259fc94f5ff..f38629b30cc9 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi - nifi-pulsar-client-services + nifi-pulsar-bundle 1.6.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml index 5c3d3a6efc49..76e4cf87aae3 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi - nifi-pulsar-client-services + nifi-pulsar-bundle 1.6.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml index 888316c60276..67a4aed10af4 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml @@ -32,6 +32,7 @@ nifi-pulsar-nar nifi-pulsar-client-service-api nifi-pulsar-client-service + nifi-pulsar-client-service-nar From e61bc9b559cac45200159efa6d875d68a5713cfa Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 9 Mar 2018 22:29:43 -0800 Subject: [PATCH 12/16] Added nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/ module --- .../nifi-pulsar-client-service-nar/pom.xml | 35 + .../src/main/resources/META-INF/LICENSE | 329 ++++++++++ .../src/main/resources/META-INF/NOTICE | 612 ++++++++++++++++++ 3 files changed, 976 insertions(+) create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/LICENSE create mode 100644 nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/NOTICE diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml new file mode 100644 index 000000000000..b8a2aa24d775 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml @@ -0,0 +1,35 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-pulsar-bundle + 1.6.0-SNAPSHOT + + + nifi-pulsar-client-service-nar + nar + + + + org.apache.nifi + nifi-pulsar-client-service-api + 1.6.0-SNAPSHOT + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 000000000000..9efc25327ca6 --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,329 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + +APACHE NIFI SUBCOMPONENTS: + +The Apache NiFi project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + +The binary distribution of this product bundles 'ANTLR 4' which is available + under a "3-clause BSD" license. For details see http://www.antlr.org/license.html + + Copyright (c) 2012 Terence Parr and Sam Harwell + All rights reserved. + Redistribution and use in source and binary forms, with or without modification, are permitted + provided that the following conditions are met: + + Redistributions of source code must retain the above copyright notice, this list of + conditions and the following disclaimer. + Redistributions in binary form must reproduce the above copyright notice, this list of + conditions and the following disclaimer in the documentation and/or other materials + provided with the distribution. + + Neither the name of the author nor the names of its contributors may be used to endorse + or promote products derived from this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY + EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF + MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL + THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, + PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF + THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles 'ParaNamer' + which is available under a BSD style license. + + Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions + are met: + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + 3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles 'icu4j' + which is available under the ICU License. + + COPYRIGHT AND PERMISSION NOTICE + + Copyright (c) 1995-2011 International Business Machines Corporation and others + + All rights reserved. + + Permission is hereby granted, free of charge, to any person obtaining a copy of this software + and associated documentation files (the "Software"), to deal in the Software without restriction, + including without limitation the rights to use, copy, modify, merge, publish, distribute, and/or + sell copies of the Software, and to permit persons to whom the Software is furnished to do so, + provided that the above copyright notice(s) and this permission notice appear in all copies of + the Software and that both the above copyright notice(s) and this permission notice appear in + supporting documentation. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING + BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT OF THIRD PARTY RIGHTS. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED + IN THIS NOTICE BE LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, OR ANY + DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, + NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE + OF THIS SOFTWARE. + + Except as contained in this notice, the name of a copyright holder shall not be used in advertising + or otherwise to promote the sale, use or other dealings in this Software without prior written + authorization of the copyright holder. + +The binary distribution of this product bundles 'Scala Library, Scala Parser, Scala Reflect' which is available + under a "3-clause BSD" license. For details see https://github.com/scala/scala/blob/v2.11.7/doc/LICENSE.md) + + This software includes projects with other licenses -- see `doc/LICENSE.md`. + + Copyright (c) 2002-2017 EPFL + Copyright (c) 2011-2017 Lightbend, Inc. + + All rights reserved. + + Redistribution and use in source and binary forms, with or without modification, + are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of the EPFL nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR + CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, + PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR + PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF + LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING + NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/NOTICE new file mode 100644 index 000000000000..1ac342e748ef --- /dev/null +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,612 @@ +nifi-druid-controller-service-api-nar +Copyright 2015-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +****************** +Apache Software License v2 +****************** + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Apache HttpComponents Client + The following NOTICE information applies: + Apache HttpComponents Client + Copyright 1999-2016 The Apache Software Foundation + + (ASLv2) Jackson JSON processor + The following NOTICE information applies: + # Jackson JSON processor + + Jackson is a high-performance, Free/Open Source JSON processing library. + It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has + been in development since 2007. + It is currently developed by a community of developers, as well as supported + commercially by FasterXML.com. + + ## Licensing + + Jackson core and extension components may licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + ## Credits + + A list of contributors may be found from CREDITS file, which is included + in some artifacts (usually source distributions); but is always available + from the source code management (SCM) system project uses. + + (ASLv2) bytebuffer-collections + The following NOTICE information applies: + bytebuffer-collections + Copyright 2011-2015 Metamarkets Group Inc. + + (ASLv2) Apache Commons IO + The following NOTICE information applies: + Apache Commons IO + Copyright 2002-2016 The Apache Software Foundation + + (ASLv2) Apache Commons Collections + The following NOTICE information applies: + Apache Commons Collections + Copyright 2001-2013 The Apache Software Foundation + + (ASLv2) Apache Commons Lang + The following NOTICE information applies: + Apache Commons Lang + Copyright 2001-2014 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + (ASLv2) Apache Commons Codec + The following NOTICE information applies: + Apache Commons Codec + Copyright 2002-2014 The Apache Software Foundation + + src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java + contains test data from http://aspell.net/test/orig/batch0.tab. + Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + + =============================================================================== + + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright: + Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + + (ASLv2) Apache Commons Logging + The following NOTICE information applies: + Apache Commons Logging + Copyright 2003-2013 The Apache Software Foundation + + (ASLv2) Apache Commons DBCP + The following NOTICE information applies: + Apache Commons DBCP + Copyright 2001-2015 The Apache Software Foundation. + + (ASLv2) Apache Commons Pool + The following NOTICE information applies: + Apache Commons Pool + Copyright 1999-2009 The Apache Software Foundation. + + (ASLv2) Apache Curator + The following NOTICE information applies: + Curator Framework + Copyright 2011-2014 The Apache Software Foundation + + Curator Client + Copyright 2011-2014 The Apache Software Foundation + + Curator Recipes + Copyright 2011-2014 The Apache Software Foundation + + Curator X-Discovery + Copyright 2013-2014 The Apache Software Foundation + + (ASLv2) Druid + The following NOTICE information applies: + Druid - a distributed column store. + Copyright 2012-2016 Metamarkets Group Inc. + Copyright 2015-2016 Yahoo! Inc. + Copyright 2015-2016 Imply Data, Inc. + + ------------------------------------------------------------------------------- + + This product contains a modified version of Andrew Duffy's java-alphanum library + * LICENSE: + * https://github.com/amjjd/java-alphanum/blob/5c036e2e492cc7f3b7bcdebd46b8f9e2a87927e5/LICENSE.txt (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/amjjd/java-alphanum + + This product contains conjunctive normal form conversion code and a variance aggregator algorithm adapted from Apache Hive + * LICENSE: + * https://github.com/apache/hive/blob/branch-2.0/LICENSE (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/apache/hive + + This product contains variable length long deserialization code adapted from Apache Lucene + * LICENSE: + * https://github.com/apache/lucene-solr/blob/master/lucene/LICENSE.txt (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/apache/lucene-solr + + This product contains a modified version of Metamarkets java-util library + * LICENSE: + * https://github.com/metamx/java-util/blob/master/LICENSE (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/metamx/java-util + * COMMIT TAG: + * https://github.com/metamx/java-util/commit/826021f + + This product contains a modified version of TestNG 6.8.7 + * LICENSE: + * http://testng.org/license/ (Apache License, Version 2.0) + * HOMEPAGE: + * http://testng.org/ + + This product contains a modified version of Metamarkets bytebuffer-collections library + * LICENSE: + * https://github.com/metamx/bytebuffer-collections/blob/master/LICENSE (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/metamx/bytebuffer-collections + * COMMIT TAG: + * https://github.com/metamx/bytebuffer-collections/commit/3d1e7c8 + + This product contains SQL query planning code adapted from Apache Calcite + * LICENSE: + * https://github.com/apache/calcite/blob/master/LICENSE (Apache License, Version 2.0) + * HOMEPAGE: + * https://calcite.apache.org/ + + This product contains a modified version of Metamarkets extendedset library + * LICENSE: + * https://github.com/metamx/extendedset/blob/master/LICENSE (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/metamx/extendedset + * COMMIT TAG: + * https://github.com/metamx/extendedset/commit/c9d647d + + This product contains a modified version of Alessandro Colantonio's CONCISE + (COmpressed 'N' Composable Integer SEt) library, extending the functionality of + ConciseSet to use IntBuffers. + * (c) 2010 Alessandro Colantonio + * + * + * LICENSE: + * Apache License, Version 2.0 + * HOMEPAGE: + * https://sourceforge.net/projects/concise/ + + This product contains a modified version of The Guava Authors's Closer class from Guava library: + * LICENSE: + * https://github.com/google/guava/blob/c462d69329709f72a17a64cb229d15e76e72199c/COPYING (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/google/guava + * COMMIT TAG: + * https://github.com/google/guava/blob/c462d69329709f72a17a64cb229d15e76e72199c + + This product contains code adapted from Apache Hadoop + * LICENSE: + * https://github.com/apache/hadoop/blob/trunk/LICENSE.txt (Apache License, Version 2.0) + * HOMEPAGE: + * http://hadoop.apache.org/ + + (ASLv2) Emitter + The following NOTICE information applies: + emitter + Copyright 2012-2015 Metamarkets Group Inc. + + (ASLv2) Finagle + The following NOTICE information applies: + Copyright 2017, Twitter Inc. + This software contains portions of code from other open-source projects, including: + >>>>>>>>>>>>>> + Apache Thrift + Copyright 2006-2010 The Apache Software Foundation. + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + (ASLv2) Guava + The following NOTICE information applies: + Guava + Copyright 2015 The Guava Authors + + (ASLv2) Google Guice + The following NOTICE information applies: + Google Guice - Core Library + Copyright 2006-2011 Google, Inc. + + (ASLv2) Jackson JSON processor + The following NOTICE information applies: + # Jackson JSON processor + + Jackson is a high-performance, Free/Open Source JSON processing library. + It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has + been in development since 2007. + It is currently developed by a community of developers, as well as supported + commercially by FasterXML.com. + + ## Licensing + + Jackson core and extension components may licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + ## Credits + + A list of contributors may be found from CREDITS file, which is included + in some artifacts (usually source distributions); but is always available + from the source code management (SCM) system project uses. + + (ASLv2) java-util + The following NOTICE information applies: + java-util + Copyright 2011-2017 Metamarkets Group Inc. + + (ASLv2) jdbi + The following NOTICE information applies: + This product includes software developed by Brian McCallister + + (ASLv2) Joda Time + The following NOTICE information applies: + This product includes software developed by + Joda.org (http://www.joda.org/). + + (ASLv2) Joda Convert + The following NOTICE information applies: + Joda Convert + Copyright 2010-present Stephen Colebourne + + This product includes software developed by + Joda.org (http://www.joda.org/). + + + Joda-Convert includes code from Google Guava, which is licensed as follows: + + Copyright (C) 2011 The Guava Authors + + Licensed 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. + + (ASLv2) Apache log4j + The following NOTICE information applies: + Apache log4j + Copyright 2007 The Apache Software Foundation + + (ASLv2) loglady + The following NOTICE information applies: + Copyright (c) 2013 Daniel Lundin + + (ASLv2) The Netty Project + The following NOTICE information applies: + + The Netty Project + ================= + + Please visit the Netty web site for more information: + + * http://netty.io/ + + Copyright 2014 The Netty Project + + The Netty Project 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. + + Also, please refer to each LICENSE..txt file, which is located in + the 'license' directory of the distribution file, for the license terms of the + components that this product depends on. + + ------------------------------------------------------------------------------- + This product contains the extensions to Java Collections Framework which has + been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + + This product contains a modified version of Robert Harder's Public Domain + Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + + This product contains a modified portion of 'Webbit', an event based + WebSocket and HTTP server, which can be obtained at: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + + This product contains a modified portion of 'SLF4J', a simple logging + facade for Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + + This product contains a modified portion of 'Apache Harmony', an open source + Java SE, which can be obtained at: + + * LICENSE: + * license/LICENSE.harmony.txt (Apache License 2.0) + * HOMEPAGE: + * http://archive.apache.org/dist/harmony/ + + This product contains a modified portion of 'jbzip2', a Java bzip2 compression + and decompression library written by Matthew J. Francis. It can be obtained at: + + * LICENSE: + * license/LICENSE.jbzip2.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jbzip2/ + + This product contains a modified portion of 'libdivsufsort', a C API library to construct + the suffix array and the Burrows-Wheeler transformed string for any input string of + a constant-size alphabet written by Yuta Mori. It can be obtained at: + + * LICENSE: + * license/LICENSE.libdivsufsort.txt (MIT License) + * HOMEPAGE: + * https://github.com/y-256/libdivsufsort + + This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM, + which can be obtained at: + + * LICENSE: + * license/LICENSE.jctools.txt (ASL2 License) + * HOMEPAGE: + * https://github.com/JCTools/JCTools + + This product optionally depends on 'JZlib', a re-implementation of zlib in + pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + + This product optionally depends on 'Compress-LZF', a Java library for encoding and + decoding data in LZF format, written by Tatu Saloranta. It can be obtained at: + + * LICENSE: + * license/LICENSE.compress-lzf.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/ning/compress + + This product optionally depends on 'lz4', a LZ4 Java compression + and decompression library written by Adrien Grand. It can be obtained at: + + * LICENSE: + * license/LICENSE.lz4.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jpountz/lz4-java + + This product optionally depends on 'lzma-java', a LZMA Java compression + and decompression library, which can be obtained at: + + * LICENSE: + * license/LICENSE.lzma-java.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jponge/lzma-java + + This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression + and decompression library written by William Kinney. It can be obtained at: + + * LICENSE: + * license/LICENSE.jfastlz.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jfastlz/ + + This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data + interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * https://github.com/google/protobuf + + This product optionally depends on 'Bouncy Castle Crypto APIs' to generate + a temporary self-signed X.509 certificate when the JVM does not provide the + equivalent functionality. It can be obtained at: + + * LICENSE: + * license/LICENSE.bouncycastle.txt (MIT License) + * HOMEPAGE: + * http://www.bouncycastle.org/ + + This product optionally depends on 'Snappy', a compression library produced + by Google Inc, which can be obtained at: + + * LICENSE: + * license/LICENSE.snappy.txt (New BSD License) + * HOMEPAGE: + * https://github.com/google/snappy + + This product optionally depends on 'JBoss Marshalling', an alternative Java + serialization API, which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://www.jboss.org/jbossmarshalling + + This product optionally depends on 'Caliper', Google's micro- + benchmarking framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.caliper.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/google/caliper + + This product optionally depends on 'Apache Commons Logging', a logging + framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + + This product optionally depends on 'Apache Log4J', a logging framework, which + can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + + This product optionally depends on 'Aalto XML', an ultra-high performance + non-blocking XML processor, which can be obtained at: + + * LICENSE: + * license/LICENSE.aalto-xml.txt (Apache License 2.0) + * HOMEPAGE: + * http://wiki.fasterxml.com/AaltoHome + + This product contains a modified version of 'HPACK', a Java implementation of + the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at: + + * LICENSE: + * license/LICENSE.hpack.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/twitter/hpack + + This product contains a modified portion of 'Apache Commons Lang', a Java library + provides utilities for the java.lang API, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-lang.txt (Apache License 2.0) + * HOMEPAGE: + * https://commons.apache.org/proper/commons-lang/ + + This product contains a forked and modified version of Tomcat Native + + * LICENSE: + * ASL2 + * HOMEPAGE: + * http://tomcat.apache.org/native-doc/ + * https://svn.apache.org/repos/asf/tomcat/native/ + + (ASLv2) opencsv (net.sf.opencsv:opencsv:2.3) + + (ASLv2) scala_util + The following NOTICE information applies: + Copyright 2012 Metamarkets Group Inc. + + (ASLv2) server-metrics + The following NOTICE information applies: + server-metrics + Copyright 2011-2015 Metamarkets Group Inc. + + (ASLv2) Apache ZooKeeper + The following NOTICE information applies: + Apache ZooKeeper + Copyright 2009-2012 The Apache Software Foundation + +************************ +Eclipse Public License 1.0 +************************ + +The following binary components are provided under the Eclipse Public License 1.0. See project link for details. + + (EPL 1.0)(MPL 2.0) H2 Database (com.h2database:h2:jar:1.3.176 - http://www.h2database.com/html/license.html) + + +************************ +Mozilla Public License 2.0 +************************ + +The following binary components are provided under the Mozilla Public License 2.0. See project link for details. + + (MPL 2.0) Rhino (org.mozilla:rhino:jar:1.7R5 - https://developer.mozilla.org/en-US/docs/Mozilla/Projects/Rhino/Download_Rhino) + + +************************ +Common Development and Distribution License 1.1 +************************ + +The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details. + + (CDDL 1.1) (GPL2 w/ CPE) jersey-client (org.glassfish.jersey.core:jersey-client:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-common (org.glassfish.jersey.core:jersey-common:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-container-servlet-core (org.glassfish.jersey.containers:jersey-container-servlet-core:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-entity-filtering (org.glassfish.jersey.ext:jersey-entity-filtering:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-hk2 (org.glassfish.jersey.inject:jersey-hk2:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-media-jaxb (org.glassfish.jersey.media:jersey-media-jaxb:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-media-json-jackson (org.glassfish.jersey.media:jersey-media-json-jackson:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-media-multipart (org.glassfish.jersey.media:jersey-media-multipart:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (org.glassfish.jersey.core:jersey-server:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) hk2 (org.glassfish.hk2:hk2:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) hk2-api (org.glassfish.hk2:hk2-api:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) hk2-utils (org.glassfish.hk2:hk2-utils:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) hk2-locator (org.glassfish.hk2:hk2-locator:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) hk2-config (org.glassfish.hk2:hk2-config:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) hk2-core (org.glassfish.hk2:hk2-core:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) hk2-runlevel (org.glassfish.hk2:hk2-runlevel:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) config-types (org.glassfish.hk2:config-types:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) class-model (org.glassfish.hk2:class-model:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) asm-all-repackaged (org.glassfish.hk2.external:asm-all-repackaged:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) aopalliance-repackaged (org.glassfish.hk2.external:aopalliance-repackaged:jar:2.5.0-b42 - https://javaee.github.io/glassfish/) + (CDDL 1.1) (GPL2 w/ CPE) javax.inject:1 as OSGi bundle (org.glassfish.hk2.external:javax.inject:jar:2.4.0-b25 - https://hk2.java.net/external/javax.inject) + (CDDL 1.1) (GPL2 w/ CPE) jersey-client (org.glassfish.jersey.core:jersey-client:jar:2.19 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-common (org.glassfish.jersey.core:jersey-common:jar:2.19 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-guava (org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:2.19 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-client (org.glassfish.jersey.core:jersey-client:jar:2.22.1 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-common (org.glassfish.jersey.core:jersey-common:jar:2.22.1 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-media-multipart (org.glassfish.jersey.media:jersey-media-multipart:jar:2.22.1 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-guava (org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:2.22.1 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.9 - https://jersey.java.net/jersey-client/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.9 - https://jersey.java.net/jersey-core/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:jar:1.9 - https://jersey.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.9 - https://jersey.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:jar:1.9 - https://jersey.java.net/) + + + (CDDL 1.1) (GPL2 w/ CPE) Old JAXB Runtime (com.sun.xml.bind:jaxb-impl:jar:2.2.3-1 - http://jaxb.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) Java Architecture For XML Binding (javax.xml.bind:jaxb-api:jar:2.2.2 - https://jaxb.dev.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) MIME Streaming Extension (org.jvnet.mimepull:mimepull:jar:1.9.3 - http://mimepull.java.net) + (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail) + (CDDL 1.1) (GPL2 w/ CPE) JSP Implementation (org.glassfish.web:javax.servlet.jsp:jar:2.3.2 - http://jsp.java.net) + (CDDL 1.1) (GPL2 w/ CPE) JavaServer Pages (TM) TagLib Implementation (org.glassfish.web:javax.servlet.jsp.jstl:jar:1.2.2 - http://jstl.java.net) + (CDDL 1.1) (GPL2 w/ CPE) Expression Language 3.0 (org.glassfish:javax.el:jar:3.0.0 - http://el-spec.java.net) + (CDDL 1.1) (GPL2 w/ CPE) JavaServer Pages(TM) API (javax.servlet.jsp:javax.servlet.jsp-api:jar:2.3.1 - http://jsp.java.net) + (CDDL 1.1) (GPL2 w/ CPE) Expression Language 3.0 API (javax.el:javax.el-api:jar:3.0.0 - http://uel-spec.java.net) + +***************** +Public Domain +***************** + +The following binary components are provided to the 'Public Domain'. See project link for details. + + (Public Domain) AOP Alliance 1.0 (http://aopalliance.sourceforge.net/) + +The following binary components are provided under the Creative Commons Zero license version 1.0. See project link for details. + + (CC0v1.0) JSR166e for Twitter (com.twitter:jsr166e:jar:1.1.0 - https://github.com/twitter/jsr166e) From 27d3f86315dd91fc8e61d4399caa289304bf5fd2 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 9 Mar 2018 22:41:34 -0800 Subject: [PATCH 13/16] Added dependency versions that appveyor complained about --- .../nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml | 3 +++ .../nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml | 2 ++ 2 files changed, 5 insertions(+) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml index 76e4cf87aae3..5caed84b68a4 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/pom.xml @@ -39,6 +39,8 @@ org.apache.nifi nifi-processor-utils + 1.6.0-SNAPSHOT + provided org.apache.nifi @@ -48,6 +50,7 @@ org.apache.nifi nifi-mock + 1.6.0-SNAPSHOT test diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml index 2931a50e918f..b7714d66770d 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml @@ -41,6 +41,7 @@ org.apache.nifi nifi-utils + 1.6.0-SNAPSHOT org.apache.nifi @@ -60,6 +61,7 @@ org.apache.nifi nifi-mock + 1.6.0-SNAPSHOT test From a6aff8ccbc5cc6fd87bddb3c1ffb0a7821b40e7f Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Fri, 9 Mar 2018 23:04:53 -0800 Subject: [PATCH 14/16] Added nifi-pulsar-client-service dependency to nifi-nar-bundle --- .../nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml index d986bf50fb98..dbc436c26b65 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml @@ -32,15 +32,21 @@ org.apache.nifi - nifi-pulsar-processors + nifi-pulsar-client-service-api-nar 1.6.0-SNAPSHOT - - + nar + + org.apache.nifi - nifi-pulsar-client-service-api-nar + nifi-pulsar-processors 1.6.0-SNAPSHOT - nar + + + + org.apache.nifi + nifi-pulsar-client-service + 1.6.0-SNAPSHOT From 0c186183fa1ee998a61960b2494838459a324023 Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Sat, 10 Mar 2018 07:25:00 -0800 Subject: [PATCH 15/16] Fixed Maven dependencies --- .../nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml | 2 +- nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml index b8a2aa24d775..4fe04879d438 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-nar/pom.xml @@ -21,7 +21,7 @@ 1.6.0-SNAPSHOT - nifi-pulsar-client-service-nar + nifi-pulsar-client-service-api-nar nar diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml index dbc436c26b65..b52d27982734 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-nar/pom.xml @@ -24,10 +24,6 @@ nifi-pulsar-nar nar - - true - true - From ffe77bd23136030599d066d0b49ac3555cc8e8ee Mon Sep 17 00:00:00 2001 From: David Kjerrumgaard Date: Thu, 15 Mar 2018 13:44:56 -0700 Subject: [PATCH 16/16] Fixed Issues identified during code review --- .../nifi-pulsar-client-service-api/pom.xml | 6 +- .../apache/nifi/pulsar/PulsarClientPool.java | 21 +++++ .../apache/nifi/pulsar/PulsarConsumer.java | 14 ++- .../nifi/pulsar/pool/PoolableResource.java | 18 ++++ .../pulsar/pool/PulsarConsumerFactory.java | 8 +- .../pulsar/pool/ResourceExceptionHandler.java | 13 +++ .../nifi/pulsar/pool/ResourceFactory.java | 16 ++++ .../nifi/pulsar/StandardPulsarClientPool.java | 28 +++--- .../nifi/pulsar/pool/ResourcePoolImpl.java | 8 +- .../TestStandardPulsarClientService.java | 8 +- .../pulsar/pool/MockPulsarClientService.java | 2 +- .../nifi-pulsar-processors/pom.xml | 6 +- ...Pulsar_1_0.java => ConsumePulsar_1_X.java} | 85 ++++++++++--------- ...Pulsar_1_0.java => PublishPulsar_1_X.java} | 63 ++++++-------- .../org.apache.nifi.processor.Processor | 4 +- .../pulsar/AbstractPulsarProcessorTest.java | 2 +- .../pulsar/ConsumePulsarProcessorTest.java | 24 +++--- .../pulsar/MockPulsarClientService.java | 2 +- .../pulsar/PublishPulsarProcessorTest.java | 44 +++++----- 19 files changed, 219 insertions(+), 153 deletions(-) rename nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/{ConsumePulsar_1_0.java => ConsumePulsar_1_X.java} (88%) rename nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/{PublishPulsar_1_0.java => PublishPulsar_1_X.java} (90%) diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml index f38629b30cc9..c3d99b4d139e 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml @@ -32,9 +32,9 @@ provided - org.apache.pulsar - pulsar-client - 1.21.0-incubating + org.apache.pulsar + pulsar-client + 1.21.0-incubating diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java index abd51f5b03f8..ee7728fb194c 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarClientPool.java @@ -25,9 +25,30 @@ @Tags({"Pulsar"}) @CapabilityDescription("Provides the ability to create Pulsar Producer / Consumer instances on demand, based on the configuration." + "properties defined") +/** + * Service definition for apache Pulsar Client ControllerService + * responsible for maintaining a pool of @PulsarProducer and + * @PulsarConsumer objects. + * + * Since both of these objects can be reused, in a manner similar + * to database connections, and the cost to create these objects is + * relatively high. The PulsarClientPool keeps these objects in pools + * for re-use. + * + * @author david + * + */ public interface PulsarClientPool extends ControllerService { + /** + * Returns the pool of @PulsarProducer objects. + * @return ResourcePool + */ public ResourcePool getProducerPool(); + /** + * Returns the pool of @PulsarConsumer objects. + * @return ResourcePool + */ public ResourcePool getConsumerPool(); } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java index 34f7453637ad..b9fd9ed2c8ad 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/PulsarConsumer.java @@ -23,17 +23,19 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@SuppressWarnings("unused") + public class PulsarConsumer implements PoolableResource { private static final Logger logger = LoggerFactory.getLogger(PulsarConsumer.class); private final Consumer consumer; + private String pulsarBrokerRootUrl; private final String topicName; private final String subscriptionName; private boolean closed = false; - public PulsarConsumer(Consumer consumer, String topic, String subscription) throws PulsarClientException { + public PulsarConsumer(Consumer consumer, String pulsarBrokerRootUrl, String topic, String subscription) throws PulsarClientException { + this.pulsarBrokerRootUrl = pulsarBrokerRootUrl; this.consumer = consumer; this.topicName = topic; this.subscriptionName = subscription; @@ -65,4 +67,12 @@ public ConsumerStats getStats() { return this.consumer.getStats(); } + public String getTransitURL() { + StringBuffer sb = new StringBuffer(); + sb.append(pulsarBrokerRootUrl).append("/") + .append(topicName).append("/") + .append(subscriptionName); + + return sb.toString(); + } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java index 4042e808b7e5..352cc2aacd0b 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PoolableResource.java @@ -16,10 +16,28 @@ */ package org.apache.nifi.pulsar.pool; +/** + * Service interface for any object that can be pooled for re-use., which + * defines methods for closing the object, effectively marking it no longer + * usable. + * + * @author david + * + */ public interface PoolableResource { + /** + * Closes the object, marking it as no longer usable. + * Typically this is called if interactions with the + * object have resulted in some sort of communication error. + */ public void close(); + /** + * Check to see if the object is usable. + * + * @return true if the close method on this object has been called. + */ public boolean isClosed(); } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java index 5101c5f4b2ad..921aac220f37 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/PulsarConsumerFactory.java @@ -30,9 +30,11 @@ public class PulsarConsumerFactory implements ResourceFactory { public static final String CONSUMER_CONFIG = "Consumer-Configuration"; private PulsarClient client; + private String pulsarBrokerRootUrl; - public PulsarConsumerFactory(PulsarClient client) { + public PulsarConsumerFactory(PulsarClient client, String pulsarBrokerRootUrl) { this.client = client; + this.pulsarBrokerRootUrl = pulsarBrokerRootUrl; } @Override @@ -44,8 +46,8 @@ public PulsarConsumer create(Properties props) throws ResourceCreationException try { // If we have a ProducerConfiguration then use it, otherwise a topic name will suffice - return (config == null) ? new PulsarConsumer(client.subscribe(topic, subscription), topic, subscription) : - new PulsarConsumer(client.subscribe(topic, subscription, config), topic, subscription); + return (config == null) ? new PulsarConsumer(client.subscribe(topic, subscription), pulsarBrokerRootUrl, topic, subscription) : + new PulsarConsumer(client.subscribe(topic, subscription, config), pulsarBrokerRootUrl, topic, subscription); } catch (PulsarClientException e) { throw new ResourceCreationException(e); diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java index f116f257896b..4108a94b1580 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceExceptionHandler.java @@ -17,7 +17,20 @@ */ package org.apache.nifi.pulsar.pool; +/** + * Wrapper interface for an exception handler that will be invoked + * in the event of any exception occurring while interacting with + * a @PoolableResource + * + * @author david + * + */ public interface ResourceExceptionHandler { + /** + * Method to be called in the event of any exception occurring while interacting + * a @PoolableResource + * @param exc The Exception thrown by the @PoolableResource + */ void handle(Exception exc); } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java index 9ed21750c812..b96a516725bf 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/pool/ResourceFactory.java @@ -18,7 +18,23 @@ import java.util.Properties; +/** + * Factory pattern interface for @PoolableResource objects. Concrete implementations + * of this interface will be responsible for the creation of @PoolableResource objects + * based on the Properties passed in. + * + * @author david + * + * @param The type of @PoolableResource this factory will create + * + */ public interface ResourceFactory { + /** + * Create a @PoolableResource based on the properties passed in. + * @param props The properties used to configure the @PoolableResource + * @return @PoolableResource of type R + * @throws ResourceCreationException if the factory cannot instantiate a @PoolableResource object + */ public R create(Properties props) throws ResourceCreationException; } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java index 7502b220f7cc..a3b5b16f87f3 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientPool.java @@ -184,32 +184,29 @@ protected List getSupportedPropertyDescriptors() { } /** - * @param context - * the configuration context - * @throws InitializationException - * if unable to create a database connection + * @param context the configuration context + * @throws InitializationException if unable to create a database connection + * @throws UnsupportedAuthenticationException if the Broker URL uses a non-supported authentication mechanism */ @OnEnabled - public void onEnabled(final ConfigurationContext context) throws InitializationException { + public void onEnabled(final ConfigurationContext context) throws InitializationException, UnsupportedAuthenticationException { createClient(context); - if (this.client == null) + if (this.client == null) { throw new InitializationException("Unable to create Pulsar Client"); + } producers = new ResourcePoolImpl(new PulsarProducerFactory(client), context.getProperty(MAX_PRODUCERS).asInteger()); - consumers = new ResourcePoolImpl(new PulsarConsumerFactory(client), context.getProperty(MAX_CONSUMERS).asInteger()); + consumers = new ResourcePoolImpl(new PulsarConsumerFactory(client, + buildPulsarBrokerRootUrl(context.getProperty(PULSAR_SERVICE_URL).getValue(), getClientConfig(context).isUseTls())), + context.getProperty(MAX_CONSUMERS).asInteger()); } private void createClient(final ConfigurationContext context) throws InitializationException { - // We can't create a client without a service URL. - if (!context.getProperty(PULSAR_SERVICE_URL).isSet()) { - return; - } - - try { + try { this.client = PulsarClient.create(buildPulsarBrokerRootUrl(context.getProperty(PULSAR_SERVICE_URL).getValue(), getClientConfig(context).isUseTls()), getClientConfig(context)); @@ -223,8 +220,9 @@ private static String buildPulsarBrokerRootUrl(String uri, boolean tlsEnabled) { StringBuilder builder = new StringBuilder(); builder.append("pulsar"); - if (tlsEnabled) - builder.append("+ssl"); + if (tlsEnabled) { + builder.append("+ssl"); + } builder.append("://"); builder.append(uri); diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java index 441879630f0d..bb183956e3f3 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/pool/ResourcePoolImpl.java @@ -53,11 +53,10 @@ public ResourcePoolImpl(ResourceFactory factory, ResourceExceptionHandler han private R createResource(Properties props) { R resource = null; try { - resource = factory.create(props); - - if (resource == null) + if (resource == null) { throw new ResourceCreationException("Unable to create resource"); + } } catch (Exception e) { resourceExceptionHandler.handle(e); @@ -113,8 +112,9 @@ public void evict(R resource) { try { // Attempt to close the connection - if (!resource.isClosed()) + if (!resource.isClosed()) { resource.close(); + } pool.removeElement(resource); --max_resources; diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java index 4b9982df30db..382f8dfbab3c 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/TestStandardPulsarClientService.java @@ -19,16 +19,10 @@ import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; -import org.junit.Before; import org.junit.Test; public class TestStandardPulsarClientService { - @Before - public void init() { - - } - @Test public void testService() throws InitializationException { final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class); @@ -36,7 +30,7 @@ public void testService() throws InitializationException { runner.addControllerService("test-good", service); runner.setProperty(service, StandardPulsarClientPool.PULSAR_SERVICE_URL, "localhost:6667"); - // runner.enableControllerService(service); + runner.enableControllerService(service); runner.assertValid(service); } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java index 2a3a5517bd25..201f9f3d8bfe 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/test/java/org/apache/nifi/pulsar/pool/MockPulsarClientService.java @@ -128,7 +128,7 @@ public PulsarConsumer getConsumer(Properties props) { String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); try { - return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + return new PulsarConsumer(mockClient.subscribe(topic, subscription), "", topic, subscription); } catch (PulsarClientException e) { return null; } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml index b7714d66770d..da5980232614 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/pom.xml @@ -54,9 +54,9 @@ provided - org.apache.pulsar - pulsar-client - 1.21.0-incubating + org.apache.pulsar + pulsar-client + 1.21.0-incubating org.apache.nifi diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_X.java similarity index 88% rename from nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java rename to nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_X.java index eccd2903f8f5..d4e47314b0b4 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_0.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/ConsumePulsar_1_X.java @@ -58,7 +58,7 @@ @CapabilityDescription("Consumes messages from Apache Pulsar " + "The complementary NiFi processor for sending messages is PublishPulsar.") @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) -public class ConsumePulsar_1_0 extends AbstractPulsarProcessor { +public class ConsumePulsar_1_X extends AbstractPulsarProcessor { static final AllowableValue EXCLUSIVE = new AllowableValue("Exclusive", "Exclusive", "There can be only 1 consumer on the same topic with the same subscription name"); static final AllowableValue SHARED = new AllowableValue("Shared", "Shared", "Multiple consumer will be able to use the same subscription name and the messages"); @@ -185,21 +185,21 @@ protected List getSupportedPropertyDescriptors() { @OnScheduled public void init(ProcessContext context) { - pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); - completionService = new ExecutorCompletionService<>(pool); + pool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger()); + completionService = new ExecutorCompletionService<>(pool); } @OnUnscheduled public void shutDown() { - // Stop all the async consumers - pool.shutdownNow(); + // Stop all the async consumers + pool.shutdownNow(); } @Override public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { try { - if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + if (context.getProperty(ASYNC_ENABLED).asBoolean()) { // Launch consumers consumeAsync(context, session); @@ -225,18 +225,18 @@ private void handleAsync(ProcessContext context, ProcessSession session) { if (msg != null) { FlowFile flowFile = null; - final byte[] value = msg.getData(); - if (value != null && value.length > 0) { - flowFile = session.create(); - flowFile = session.write(flowFile, out -> { - out.write(value); - }); - } - - session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); - session.transfer(flowFile, REL_SUCCESS); - session.commit(); - getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); + final byte[] value = msg.getData(); + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + } + + session.getProvenanceReporter().receive(flowFile, "From " + getWrappedConsumer(context).getTransitURL()); + session.transfer(flowFile, REL_SUCCESS); + session.commit(); + getWrappedConsumer(context).getConsumer().acknowledgeAsync(msg); } } catch (InterruptedException | ExecutionException | PulsarClientException e) { @@ -251,9 +251,9 @@ public void close(final ProcessContext context) { getLogger().info("Disconnecting Pulsar Consumer"); if (consumer != null) { - context.getProperty(PULSAR_CLIENT_SERVICE) - .asControllerService(PulsarClientPool.class) - .getConsumerPool().evict(consumer); + context.getProperty(PULSAR_CLIENT_SERVICE) + .asControllerService(PulsarClientPool.class) + .getConsumerPool().evict(consumer); } consumer = null; @@ -266,14 +266,14 @@ public void close(final ProcessContext context) { */ private void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException { - Consumer consumer = getWrappedConsumer(context).getConsumer(); + Consumer consumer = getWrappedConsumer(context).getConsumer(); - completionService.submit(new Callable() { - @Override - public Message call() throws Exception { - return consumer.receiveAsync().get(); - } - }); + completionService.submit(new Callable() { + @Override + public Message call() throws Exception { + return consumer.receiveAsync().get(); + } + }); } @@ -293,19 +293,19 @@ private void consume(ProcessContext context, ProcessSession session) throws Puls try { - msg = consumer.receive(); - final byte[] value = msg.getData(); + msg = consumer.receive(); + final byte[] value = msg.getData(); - if (value != null && value.length > 0) { - flowFile = session.create(); - flowFile = session.write(flowFile, out -> { - out.write(value); - }); + if (value != null && value.length > 0) { + flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + out.write(value); + }); - session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); + session.getProvenanceReporter().receive(flowFile, "From " + context.getProperty(TOPIC).getValue()); session.transfer(flowFile, REL_SUCCESS); logger.info("Created {} from {} messages received from Pulsar Server and transferred to 'success'", - new Object[]{flowFile, 1}); + new Object[]{flowFile, 1}); session.commit(); @@ -324,10 +324,10 @@ private void consume(ProcessContext context, ProcessSession session) throws Puls getLogger().info("Acknowledging message " + msg.getMessageId()); consumer.acknowledge(msg); - } else { - // We didn't consume any data, so - session.commit(); - } + } else { + // We didn't consume any data, so + session.commit(); + } } catch (PulsarClientException e) { context.yield(); @@ -338,8 +338,9 @@ private void consume(ProcessContext context, ProcessSession session) throws Puls private PulsarConsumer getWrappedConsumer(ProcessContext context) throws PulsarClientException { - if (consumer != null) + if (consumer != null) { return consumer; + } final PulsarClientPool pulsarClientService = context.getProperty(PULSAR_CLIENT_SERVICE) .asControllerService(PulsarClientPool.class); diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_X.java similarity index 90% rename from nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java rename to nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_X.java index d78aa0323622..65e40bdfe272 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_0.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/PublishPulsar_1_X.java @@ -16,8 +16,7 @@ */ package org.apache.nifi.processors.pulsar; -import java.io.IOException; -import java.io.InputStream; +import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -39,14 +38,12 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.pulsar.PulsarClientPool; import org.apache.nifi.pulsar.PulsarProducer; import org.apache.nifi.pulsar.cache.LRUCache; import org.apache.nifi.pulsar.pool.PulsarProducerFactory; import org.apache.nifi.pulsar.pool.ResourcePool; -import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.util.StringUtils; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.MessageId; @@ -56,14 +53,14 @@ import org.apache.pulsar.client.api.PulsarClientException; @Tags({"Apache", "Pulsar", "Put", "Send", "Message", "PubSub"}) -@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Pulsar using the Pulsar 1.21 Producer API." +@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Pulsar using the Pulsar 1.X Producer API." + "The messages to send may be individual FlowFiles or may be delimited, using a " + "user-specified delimiter, such as a new-line. " - + "The complementary NiFi processor for fetching messages is ConsumePulsar.") + + "The complementary NiFi processor for fetching messages is ConsumePulsar_1_X.") @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Pulsar for this FlowFile. This attribute is added only to " + "FlowFiles that are routed to success.") -public class PublishPulsar_1_0 extends AbstractPulsarProcessor { +public class PublishPulsar_1_X extends AbstractPulsarProcessor { protected static final String MSG_COUNT = "msg.count"; @@ -214,8 +211,9 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro FlowFile flowFile = session.get(); - if (flowFile == null) + if (flowFile == null) { return; + } final ComponentLog logger = getLogger(); final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); @@ -227,14 +225,10 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro } // Read the contents of the FlowFile into a byte array - final byte[] messageContent = new byte[(int) flowFile.getSize()]; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - StreamUtils.fillBuffer(in, messageContent, true); - } - }); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + session.exportTo(flowFile, baos); + final byte[] messageContent = baos.toByteArray(); // Nothing to do, so skip this Flow file. if (messageContent == null || messageContent.length < 1) { session.transfer(flowFile, REL_SUCCESS); @@ -245,7 +239,7 @@ public void process(final InputStream in) throws IOException { Producer producer = getWrappedProducer(topic, context).getProducer(); - if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) { + if (context.getProperty(ASYNC_ENABLED).asBoolean()) { this.sendAsync(producer, session, flowFile, messageContent); } else { this.send(producer, session, flowFile, messageContent); @@ -262,13 +256,13 @@ public void process(final InputStream in) throws IOException { private void send(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) throws PulsarClientException { - MessageId msgId = producer.send(messageContent); + MessageId msgId = producer.send(messageContent); - if (msgId != null) { - flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); - session.adjustCounter("Messages Sent", 1, true); - session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); - session.transfer(flowFile, REL_SUCCESS); + if (msgId != null) { + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent message " + msgId + " to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); } else { session.transfer(flowFile, REL_FAILURE); @@ -278,20 +272,19 @@ private void send(Producer producer, ProcessSession session, FlowFile flowFile, private void sendAsync(Producer producer, ProcessSession session, FlowFile flowFile, byte[] messageContent) { - producer.sendAsync(messageContent).handle((msgId, ex) -> { - if (msgId != null) { - return msgId; - } else { - // TODO Communicate the error back up to the onTrigger method so we can invalidate this producer. - getLogger().warn("Problem ", ex); - return null; - } - }); + producer.sendAsync(messageContent).handle((msgId, ex) -> { + if (msgId != null) { + return msgId; + } else { + getLogger().warn("Problem ", ex); + return null; + } + }); - flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); - session.adjustCounter("Messages Sent", 1, true); - session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); - session.transfer(flowFile, REL_SUCCESS); + flowFile = session.putAttribute(flowFile, MSG_COUNT, "1"); + session.adjustCounter("Messages Sent", 1, true); + session.getProvenanceReporter().send(flowFile, "Sent async message to " + producer.getTopic() ); + session.transfer(flowFile, REL_SUCCESS); } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index 88f24ff1cf75..e2e5b6d0e9a1 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -12,5 +12,5 @@ # 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. -org.apache.nifi.processors.pulsar.ConsumePulsar_1_0 -org.apache.nifi.processors.pulsar.PublishPulsar_1_0 +org.apache.nifi.processors.pulsar.ConsumePulsar_1_X +org.apache.nifi.processors.pulsar.PublishPulsar_1_X diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java index 9110aaf85452..720f17975ba7 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessorTest.java @@ -32,6 +32,6 @@ protected void addPulsarClientService() throws InitializationException { final MockPulsarClientService pulsarClient = new MockPulsarClientService(mockClient); runner.addControllerService("pulsarClient", pulsarClient); runner.enableControllerService(pulsarClient); - runner.setProperty(PublishPulsar_1_0.PULSAR_CLIENT_SERVICE, "pulsarClient"); + runner.setProperty(PublishPulsar_1_X.PULSAR_CLIENT_SERVICE, "pulsarClient"); } } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java index 8ed0a2480b8c..2bd1ebf9e750 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/ConsumePulsarProcessorTest.java @@ -53,7 +53,7 @@ public class ConsumePulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { - runner = TestRunners.newTestRunner(ConsumePulsar_1_0.class); + runner = TestRunners.newTestRunner(ConsumePulsar_1_X.class); mockClient = mock(PulsarClient.class); mockConsumer = mock(Consumer.class); mockMessage = mock(Message.class); @@ -80,10 +80,10 @@ public void init() throws InitializationException { public void emptyMessageTest() { when(mockMessage.getData()).thenReturn("".getBytes()); - runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); + runner.setProperty(ConsumePulsar_1_X.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_X.SUBSCRIPTION, "bar"); runner.run(); - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_X.REL_SUCCESS); } @Test @@ -113,10 +113,10 @@ public void multipleAsyncMessagesTest() throws PulsarClientException { public void onStoppedTest() throws NoSuchMethodException, SecurityException, PulsarClientException { when(mockMessage.getData()).thenReturn("Mocked Message".getBytes()); - runner.setProperty(ConsumePulsar_1_0.TOPIC, "foo"); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, "bar"); + runner.setProperty(ConsumePulsar_1_X.TOPIC, "foo"); + runner.setProperty(ConsumePulsar_1_X.SUBSCRIPTION, "bar"); runner.run(10, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_X.REL_SUCCESS); runner.assertQueueEmpty(); @@ -135,14 +135,14 @@ private void sendMessages(String msg, String topic, String sub, boolean async, i when(mockMessage.getData()).thenReturn(msg.getBytes()); - runner.setProperty(ConsumePulsar_1_0.ASYNC_ENABLED, Boolean.toString(async)); - runner.setProperty(ConsumePulsar_1_0.TOPIC, topic); - runner.setProperty(ConsumePulsar_1_0.SUBSCRIPTION, sub); + runner.setProperty(ConsumePulsar_1_X.ASYNC_ENABLED, Boolean.toString(async)); + runner.setProperty(ConsumePulsar_1_X.TOPIC, topic); + runner.setProperty(ConsumePulsar_1_X.SUBSCRIPTION, sub); runner.run(itertions, true); - runner.assertAllFlowFilesTransferred(ConsumePulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(ConsumePulsar_1_X.REL_SUCCESS); - List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_0.REL_SUCCESS); + List flowFiles = runner.getFlowFilesForRelationship(ConsumePulsar_1_X.REL_SUCCESS); assertEquals(itertions, flowFiles.size()); for (MockFlowFile ff : flowFiles) { diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java index 54cbae24f17d..ba2d29ae08b5 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/MockPulsarClientService.java @@ -138,7 +138,7 @@ public PulsarConsumer getConsumer(Properties props) { String topic = props.getProperty(PulsarConsumerFactory.TOPIC_NAME); String subscription = props.getProperty(PulsarConsumerFactory.SUBSCRIPTION_NAME); try { - return new PulsarConsumer(mockClient.subscribe(topic, subscription), topic, subscription); + return new PulsarConsumer(mockClient.subscribe(topic, subscription), "", topic, subscription); } catch (PulsarClientException e) { return null; } diff --git a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java index 6db8d7724a0c..6bf76e874b76 100644 --- a/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java +++ b/nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/test/java/org/apache/nifi/processors/pulsar/PublishPulsarProcessorTest.java @@ -48,7 +48,7 @@ public class PublishPulsarProcessorTest extends AbstractPulsarProcessorTest { @Before public void init() throws InitializationException { - runner = TestRunners.newTestRunner(PublishPulsar_1_0.class); + runner = TestRunners.newTestRunner(PublishPulsar_1_X.class); mockClient = mock(PulsarClient.class); mockProducer = mock(Producer.class); @@ -86,15 +86,15 @@ public boolean matches(Object argument) { @Test public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); + runner.setProperty(PublishPulsar_1_X.TOPIC, "${topic}"); final String content = "some content"; Map attributes = new HashMap(); - attributes.put(PublishPulsar_1_0.TOPIC.getName(), ""); + attributes.put(PublishPulsar_1_X.TOPIC.getName(), ""); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_FAILURE); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_X.REL_FAILURE); // Confirm that no Producer as created verify(mockClient, times(0)).createProducer(anyString()); @@ -103,15 +103,15 @@ public void invalidTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar_1_0.TOPIC, "${topic}"); + runner.setProperty(PublishPulsar_1_X.TOPIC, "${topic}"); final String content = "some content"; Map attributes = new HashMap(); - attributes.put(PublishPulsar_1_0.TOPIC.getName(), "topic-b"); + attributes.put(PublishPulsar_1_X.TOPIC.getName(), "topic-b"); runner.enqueue(content.getBytes("UTF-8"), attributes ); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_X.REL_SUCCESS); // Verify that we sent the data to topic-b. verify(mockClient, times(1)).createProducer("topic-b"); @@ -119,14 +119,14 @@ public void dynamicTopicTest() throws UnsupportedEncodingException, PulsarClient @Test public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_X.TOPIC, "my-topic"); final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_X.REL_SUCCESS); - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_X.REL_SUCCESS).get(0); outFile.assertContentEquals(content); // Verify that we sent the data to my-topic. @@ -139,15 +139,15 @@ public void singleFlowFileTest() throws UnsupportedEncodingException, PulsarClie @Test public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_X.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_X.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_X.REL_SUCCESS); - final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_0.REL_SUCCESS).get(0); + final MockFlowFile outFile = runner.getFlowFilesForRelationship(PublishPulsar_1_X.REL_SUCCESS).get(0); outFile.assertContentEquals(content); @@ -161,14 +161,14 @@ public void singleFlowFileAsyncTest() throws UnsupportedEncodingException, Pulsa @Test public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-topic"); + runner.setProperty(PublishPulsar_1_X.TOPIC, "my-topic"); final String content = "some content"; // Hack, since runner.run(20, false); doesn't work as advertised for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_X.REL_SUCCESS); } @@ -179,8 +179,8 @@ public void multipleFlowFilesTest() throws UnsupportedEncodingException, PulsarC @Test public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, PulsarClientException { - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_X.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_X.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; @@ -188,7 +188,7 @@ public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, Pu for (int idx = 0; idx < 20; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_X.REL_SUCCESS); } // Verify that the send method on the producer was called with the expected content @@ -197,15 +197,15 @@ public void multipleFlowFilesAsyncTest() throws UnsupportedEncodingException, Pu @Test public void stressTest() throws UnsupportedEncodingException { - runner.setProperty(PublishPulsar_1_0.TOPIC, "my-async-topic"); - runner.setProperty(PublishPulsar_1_0.ASYNC_ENABLED, Boolean.TRUE.toString()); + runner.setProperty(PublishPulsar_1_X.TOPIC, "my-async-topic"); + runner.setProperty(PublishPulsar_1_X.ASYNC_ENABLED, Boolean.TRUE.toString()); final String content = "some content"; for (int idx = 0; idx < 9999; idx++) { runner.enqueue(content.getBytes("UTF-8")); runner.run(); - runner.assertAllFlowFilesTransferred(PublishPulsar_1_0.REL_SUCCESS); + runner.assertAllFlowFilesTransferred(PublishPulsar_1_X.REL_SUCCESS); } }