You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by david-streamlio <gi...@git.apache.org> on 2018/05/14 22:03:05 UTC

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

GitHub user david-streamlio opened a pull request:

    https://github.com/apache/nifi/pull/2702

    Added Apache Pulsar processors

    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [ ] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    
    - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [ ] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [ ] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [ ] Have you written or updated unit tests to verify your changes?
    - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
    - [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
    - [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/david-streamlio/nifi NIFI-4914

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/2702.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2702
    
----
commit f67fb157b9e6f04c8b913fe30b815b181eebf53a
Author: David Kjerrumgaard <da...@...>
Date:   2018-05-14T21:30:50Z

    Added Apache Pulsar processors

----


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    Just wanted to check in to see if there is anything more I needed to do on my end, or if the testing instructions were clear enough.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197461468
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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"
    --- End diff --
    
    Can you describe the trade-offs here? Does one value increase the risk of data duplication? Data loss? Performance?


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197455824
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    --- End diff --
    
    These member variables are not protected and thus the Processor is not thread-safe. Need to ensure that these are appropriately protected.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    Don't worry about others' build errors for now. Master might just be a little off. TBH, we can even merge the code if the build is failing for reasons others than your commits so don't worry about that.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197786927
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/pubsub/ConsumePulsar.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.pubsub;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +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.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processors.pulsar.AbstractPulsarConsumerProcessor;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +@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 AbstractPulsarConsumerProcessor<byte[]> {
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        try {
    +            if (context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +                consumeAsync(context, session);
    +                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<Message<byte[]>> done = consumerService.take();
    --- End diff --
    
    Check if the processor itself is stopped (by calling `isRunning()`). This way, if the user clicks Stop, the processor doesn't just continue on indefinitely.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197451230
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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 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 List<PropertyDescriptor> properties;
    +    private volatile PulsarClient client;
    +    private boolean secure = false;
    +    private ClientBuilder builder;
    +
    +    static {
    +        final List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(PULSAR_SERVICE_URL);
    +        props.add(ALLOW_TLS_INSECURE_CONNECTION);
    +        props.add(CONCURRENT_LOOKUP_REQUESTS);
    +        props.add(CONNECTIONS_PER_BROKER);
    +        props.add(IO_THREADS);
    +        props.add(KEEP_ALIVE_INTERVAL);
    +        props.add(LISTENER_THREADS);
    +        props.add(MAXIMUM_LOOKUP_REQUESTS);
    +        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<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +       return properties;
    +    }
    +
    +    /**
    +     * @param context the configuration context
    +     * @throws InitializationException if unable to connect to the Pulsar Broker
    +     * @throws UnsupportedAuthenticationException if the Broker URL uses a non-supported authentication mechanism
    +     */
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) throws InitializationException, UnsupportedAuthenticationException {
    +        createClient(context);
    +
    +        if (this.client == null) {
    +            throw new InitializationException("Unable to create Pulsar Client");
    +        }
    +    }
    +
    +    private void createClient(final ConfigurationContext context) throws InitializationException {
    +        try {
    +            this.client = getClientBuilder(context).build();
    +        } 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 ClientBuilder getClientBuilder(ConfigurationContext context) throws UnsupportedAuthenticationException {
    +        if (builder == null) {
    --- End diff --
    
    We should avoid using a member variable here and just create a new Builder each time. As-is, the code is not thread-safe, and un-setting a property will result in a bug, where the builder still has the value set.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r198317582
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    --- End diff --
    
    I have updated the code to use a countdown latch for all async message handling, this ensures I have confirmation of message hand-off to the broker for ALL messages in the session, prior to routing the session to SUCCESS or FAILURE. 
    
    I also added message demarcation, as async processing of a single message doesn't make much sense, as I am forced to wait for a response from the broker before committing the session anyway. Therefore, I decided that a useful scenario would be to have async processing of a demarcated message, with a countdown latch to ensure all the messages are received before proceeding.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197519036
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    --- End diff --
    
    OK, that makes sense. Thanks for the clarification. It probably makes sense to leave an inline comment there, then, to explain the reasoning behind it.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    @MikeThomsen I've only spent a couple of minutes looking at this, but I'm not sure it can work out as nicely as you are hoping... the controller service API here is heavily dependent on the actual pulsar client API. 
    
    The only way you can get the transparent swapping between processors and CS impls is if the client API is hidden behind the CS impl. For example with HBase, we have...
    
    - HBase processors (no dependency on hbase-client, dependency on hbase CS api)
    - HBase CS API (no dependency on hbase-client)
    - HBase CS 1.1.2 impl (dependency on hbase-client 1.1.2)
    
    So because the processors and CS API do not know about hbase-client, then we can transparently provide new implementations without changing the processors.
    
    In this case we have...
    - Pulsar processors (depends on Pulsar client 1.21.0-incubating)
    - Pulsar CS API (depends on Pulsar client 1.21.0-incubating)
    - Pulsar CS Impls (depends on Pulsar client 1.21.0-incubating)
    
    I'm not saying the current setup is bad, just mentioning that it won't work out the way the hbase setup works. 
    
    The trade-off is that in order to achieve the hbase setup you essentially need to recreate parts of their client API and depending how much you have to recreate, it may not be worth it if you are recreating the entire pulsar client API just to shield the processors.
    
    Also, something to think about is whether Pulsar's client will work well across different broker versions. For example, when Pulsar 2.x comes out, will the 1.x client work well against a 2.x broker? or vice versa?
    
    In Kafka land their client has had issues across versions, like 0.8 client against 0.9 broker did not perform as well as 0.9 client against 0.9 broker, so for this reason you really need to use the corresponding client that goes with the broker.
    
    If Pulsar's client doesn't have this problem, maybe we don't need to worry at all about this versioning stuff.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197625631
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/pubsub/PublishPulsar.java ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.pubsub;
    +
    +import java.io.ByteArrayOutputStream;
    +
    +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.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.exception.ProcessException;
    +import org.apache.nifi.processors.pulsar.AbstractPulsarProducerProcessor;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.pulsar.client.api.MessageId;
    +import org.apache.pulsar.client.api.Producer;
    +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.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.")
    +@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 AbstractPulsarProducerProcessor<byte[]> {
    +
    +    @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
    +        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.putAttribute(flowFile, "msg.count", "0");
    +            session.transfer(flowFile, REL_SUCCESS);
    +            return;
    +        }
    +
    +        try {
    +            Producer<byte[]> producer = getProducer(context, topic);
    +
    +            if (context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +                sendAsync(producer, session, messageContent);
    +                session.remove(flowFile);
    --- End diff --
    
    Since we are operating in async mode at this point, If we transfer to success, we can't later transfer it to failure if there was a connection failure etc. 
    
    How would you advise I handle that scenario?  If we have already routed the Flowfile to success, how will the user be able to retry the FF again after the connection problem is resolved?


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197465732
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    +                           session.transfer(flowFile, REL_FAILURE);
    +                           return null;
    +                        }
    +                   }).get();
    +                 } catch (final Throwable t) {
    +                   getLogger().error("Unable to send message to Pulsar asyncronously.", t);
    +                   session.transfer(session.create(), REL_FAILURE);
    --- End diff --
    
    Again, why are we creating a new FlowFile and sending it to failure? In this case, the original incoming FlowFile should go to failure, not a newly-created FlowFile.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    looks like we're back in git pr funkystate with tons of non contrib commits in the PR...


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197844157
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/pubsub/ConsumePulsar.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.pubsub;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +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.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processors.pulsar.AbstractPulsarConsumerProcessor;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +@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 AbstractPulsarConsumerProcessor<byte[]> {
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        try {
    +            if (context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +                consumeAsync(context, session);
    +                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<Message<byte[]>> done = consumerService.take();
    --- End diff --
    
    There isn't any "isRunning()' method defined in the base AbstractProcessor class. The closest method is "isScheduled()" 


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197467328
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/pubsub/ConsumePulsar.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.pubsub;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +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.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processors.pulsar.AbstractPulsarConsumerProcessor;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +@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 AbstractPulsarConsumerProcessor<byte[]> {
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        try {
    +            if (context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +                consumeAsync(context, session);
    +                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<Message<byte[]>> done = consumerService.take();
    --- End diff --
    
    Should avoid calling take() and instead poll for some amount of time like 50 ms or 1 second. If nothing available, check if the processor is stopped and if so gracefully stop processing.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197495330
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    --- End diff --
    
    We need this for securing our connections, so it is an additional Cert file that is required beyond what the SSL_CONTEXT_SERVICE provides.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197906886
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    --- End diff --
    
    You'll have two options - you can either extend AbstractSessionFactoryProcessor, instead of AbstractProcessor, and manage the sessions yourself (in which case you just have to ensure that in all cases the session is properly rolled back or committed), or you could wait on a `Future`, perhaps, to avoid returning from `onTrigger` until the messages have been acknowledged.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197505730
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    --- End diff --
    
    I have updated the comment. Basically, the default setting of zero ensures exactly-once processing of the messages. Changing this to non-zero value results in at least once processing instead.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197453808
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    --- End diff --
    
    I would expect the pattern here to be applied against topic names, as an alternative to the "Topic Names" property. However, here, it looks like you're expecting the regex to match a specific URI. Can you provide more documentation as to what the pattern is going to be applied against, how to use, etc.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197508790
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    --- End diff --
    
    I added the wait after some of my unit tests showed that not all of the messages consumed where being acknowledged. Adding the delay provides some additional opportunity for the acknowledgements to occur.  By ensuring that the messages are properly acked, we prevent re-processing the same messages in the event of a shutdown and restart of the processor since the un-acked messages would be replayed on startup.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    Your local `master` is in sync with your repository (`origin`) but not the Apache GitHub repository (`upstream`). You need to do the following:
    
    ```
    $ git checkout master
    $ git pull upstream master
    $ git checkout NIFI-4914
    $ git rebase master
    ```


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197519282
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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 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 List<PropertyDescriptor> properties;
    +    private volatile PulsarClient client;
    +    private boolean secure = false;
    +    private ClientBuilder builder;
    +
    +    static {
    +        final List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(PULSAR_SERVICE_URL);
    +        props.add(ALLOW_TLS_INSECURE_CONNECTION);
    +        props.add(CONCURRENT_LOOKUP_REQUESTS);
    +        props.add(CONNECTIONS_PER_BROKER);
    +        props.add(IO_THREADS);
    +        props.add(KEEP_ALIVE_INTERVAL);
    +        props.add(LISTENER_THREADS);
    +        props.add(MAXIMUM_LOOKUP_REQUESTS);
    +        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<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +       return properties;
    +    }
    +
    +    /**
    +     * @param context the configuration context
    +     * @throws InitializationException if unable to connect to the Pulsar Broker
    +     * @throws UnsupportedAuthenticationException if the Broker URL uses a non-supported authentication mechanism
    +     */
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) throws InitializationException, UnsupportedAuthenticationException {
    +        createClient(context);
    +
    +        if (this.client == null) {
    +            throw new InitializationException("Unable to create Pulsar Client");
    +        }
    +    }
    +
    +    private void createClient(final ConfigurationContext context) throws InitializationException {
    +        try {
    +            this.client = getClientBuilder(context).build();
    +        } 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 ClientBuilder getClientBuilder(ConfigurationContext context) throws UnsupportedAuthenticationException {
    +        if (builder == null) {
    +           builder = PulsarClient.builder();
    +
    +           if (context.getProperty(ALLOW_TLS_INSECURE_CONNECTION).isSet()) {
    --- End diff --
    
    Yes - any property that has a default value cannot be cleared out. If the user unsets it, it will be reset to whatever the default value is.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r188922684
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/.gitignore ---
    @@ -0,0 +1,5 @@
    +.project
    --- End diff --
    
    You should be able to get rid of this file completely because there's a master `.gitignore`.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197446420
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 " +
    --- End diff --
    
    Again, we should not be enforcing arbitrary time units but instead allow the user to enter whatever time unit they choose.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197451506
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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 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 List<PropertyDescriptor> properties;
    +    private volatile PulsarClient client;
    +    private boolean secure = false;
    +    private ClientBuilder builder;
    +
    +    static {
    +        final List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(PULSAR_SERVICE_URL);
    +        props.add(ALLOW_TLS_INSECURE_CONNECTION);
    +        props.add(CONCURRENT_LOOKUP_REQUESTS);
    +        props.add(CONNECTIONS_PER_BROKER);
    +        props.add(IO_THREADS);
    +        props.add(KEEP_ALIVE_INTERVAL);
    +        props.add(LISTENER_THREADS);
    +        props.add(MAXIMUM_LOOKUP_REQUESTS);
    +        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<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +       return properties;
    +    }
    +
    +    /**
    +     * @param context the configuration context
    +     * @throws InitializationException if unable to connect to the Pulsar Broker
    +     * @throws UnsupportedAuthenticationException if the Broker URL uses a non-supported authentication mechanism
    +     */
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) throws InitializationException, UnsupportedAuthenticationException {
    +        createClient(context);
    +
    +        if (this.client == null) {
    --- End diff --
    
    I believe this is dead code - the only way that `this.client` would be null is if `createClient` threw an Exception, in which case we would not get here, right?


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197620107
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/pubsub/ConsumePulsar.java ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.pubsub;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +
    +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.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processors.pulsar.AbstractPulsarConsumerProcessor;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +@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 AbstractPulsarConsumerProcessor<byte[]> {
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
    +        try {
    +            if (context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +                consumeAsync(context, session);
    +                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<Message<byte[]>> done = consumerService.take();
    --- End diff --
    
    I am not sure what you are suggesting here. Should I check if the Nifi processor itself is stopped, or should I check the Pulsar consumer object to see if it has reached the end of the topic, etc?


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197451729
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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 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 List<PropertyDescriptor> properties;
    +    private volatile PulsarClient client;
    +    private boolean secure = false;
    +    private ClientBuilder builder;
    +
    +    static {
    +        final List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(PULSAR_SERVICE_URL);
    +        props.add(ALLOW_TLS_INSECURE_CONNECTION);
    +        props.add(CONCURRENT_LOOKUP_REQUESTS);
    +        props.add(CONNECTIONS_PER_BROKER);
    +        props.add(IO_THREADS);
    +        props.add(KEEP_ALIVE_INTERVAL);
    +        props.add(LISTENER_THREADS);
    +        props.add(MAXIMUM_LOOKUP_REQUESTS);
    +        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<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +       return properties;
    +    }
    +
    +    /**
    +     * @param context the configuration context
    +     * @throws InitializationException if unable to connect to the Pulsar Broker
    +     * @throws UnsupportedAuthenticationException if the Broker URL uses a non-supported authentication mechanism
    +     */
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) throws InitializationException, UnsupportedAuthenticationException {
    +        createClient(context);
    +
    +        if (this.client == null) {
    +            throw new InitializationException("Unable to create Pulsar Client");
    +        }
    +    }
    +
    +    private void createClient(final ConfigurationContext context) throws InitializationException {
    --- End diff --
    
    Is there a reason for separating this out into its own separate client? It seems like this should just be part of the `onEnabled` method.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197459647
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    +        }
    +    }
    +
    +    protected Consumer<T> getConsumer(ProcessContext context) throws PulsarClientException {
    +
    +        if (consumer == null) {
    +           ConsumerBuilder<T> builder = (ConsumerBuilder<T>) getPulsarClient(context).newConsumer();
    +
    +           if (context.getProperty(TOPICS).isSet()) {
    +              builder = builder.topic(context.getProperty(TOPICS)
    +                               .evaluateAttributeExpressions().getValue().split("[, ]"));
    +           } else if (context.getProperty(TOPICS_PATTERN).isSet()) {
    +              builder = builder.topicsPattern(context.getProperty(TOPICS_PATTERN).getValue());
    +           } else {
    +             throw new PulsarClientException("No topic specified.");
    +           }
    +
    +           if (context.getProperty(SUBSCRIPTION_NAME).isSet()) {
    --- End diff --
    
    No need to check this - Subscription Name is required. It will always be set, or we won't get to this part in the code.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio closed the pull request at:

    https://github.com/apache/nifi/pull/2702


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    Looks like there are error in the Email processors causing the CI build to fail.
    
    [ERROR] Tests run: 5, Failures: 0, Errors: 2, Skipped: 0, Time elapsed: 1.996 s <<< FAILURE! - in org.apache.nifi.processors.email.TestConsumeEmail
    [ERROR] testConsumePOP3(org.apache.nifi.processors.email.TestConsumeEmail)  Time elapsed: 1.014 s  <<< ERROR!
    java.lang.IllegalStateException: Could not start mail server imap:127.0.0.1:3143, try to set server startup timeout > 1000 via ServerSetup.setServerStartupTimeout(timeoutInMs)
    	at org.apache.nifi.processors.email.TestConsumeEmail.setUp(TestConsumeEmail.java:55)
    
    [ERROR] testConsumePOP3(org.apache.nifi.processors.email.TestConsumeEmail)  Time elapsed: 1.017 s  <<< ERROR!
    java.lang.NullPointerException
    	at org.apache.nifi.processors.email.TestConsumeEmail.cleanUp(TestConsumeEmail.java:66)
    
    [INFO] 
    [INFO] Results:
    [INFO] 
    [ERROR] Errors: 
    [ERROR] org.apache.nifi.processors.email.TestConsumeEmail.testConsumePOP3(org.apache.nifi.processors.email.TestConsumeEmail)
    [ERROR]   Run 1: TestConsumeEmail.setUp:55 » IllegalState Could not start mail server imap:127....
    [ERROR]   Run 2: TestConsumeEmail.cleanUp:66 NullPointer
    [INFO] 
    [INFO] 
    [ERROR] Tests run: 21, Failures: 0, Errors: 1, Skipped: 0


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    @markap14 I think you did a good chunk of the Kafka work, so would you like to take a look at this and join the review?


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    @david-streamlio I'd like to get back to this this week, so I have a question about your sandbox. Other than loading it up following the instructions that are provided w/ it, what needs to be done to set it up for use with these processors and services?


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197461291
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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)
    --- End diff --
    
    Should avoid using a Boolean Validator and instead use allowable values.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197851635
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    --- End diff --
    
    I am not sure to accomplish this, since after I submit the callback method to the ExecutorService, the original calling thread returns to the AbstractProcessor base class' `onTrigger()` method, which calls makes the call to `session.commit();`


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    That's correct. I would recommend you build a custom bundle and deliver that because you can deliver NARs independent of the release. Then any bug fixes you find along the way can be merged into this PR in 1.8.
    
    Also, FYI, we have two other big PRs that came in that are waiting. One's a big refactor of InfluxDB support and the other is a MarkLogic PR. Each of these tends to be the equivalent in difficult of reviewing half a dozen or more typical PRs so it can be hard to triage.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197454824
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    --- End diff --
    
    It feels like this property should have a default value (perhaps `30 secs`) - I would avoid defaulting to "no timeout".


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197448780
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 " +
    --- End diff --
    
    Again, want to avoid explicit time units


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197464924
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    --- End diff --
    
    We should probably not be waiting here.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    I tried rebase-ing against master, but it had no effect. I think that is because my "master" is a fork of the 1.7.0 branch... Anyways, here is the output of the rebase commands:
    
    "Davids-MacBook-Pro:nifi david$ git rebase master
    Current branch NIFI-4914 is up to date.
    Davids-MacBook-Pro:nifi david$ git checkout master
    Switched to branch 'master'
    Your branch is up to date with 'origin/master'."
    
    All the pom's are still using version 1.7.0-SNAPSHOT.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197498285
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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 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 List<PropertyDescriptor> properties;
    +    private volatile PulsarClient client;
    +    private boolean secure = false;
    +    private ClientBuilder builder;
    +
    +    static {
    +        final List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(PULSAR_SERVICE_URL);
    +        props.add(ALLOW_TLS_INSECURE_CONNECTION);
    +        props.add(CONCURRENT_LOOKUP_REQUESTS);
    +        props.add(CONNECTIONS_PER_BROKER);
    +        props.add(IO_THREADS);
    +        props.add(KEEP_ALIVE_INTERVAL);
    +        props.add(LISTENER_THREADS);
    +        props.add(MAXIMUM_LOOKUP_REQUESTS);
    +        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<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +       return properties;
    +    }
    +
    +    /**
    +     * @param context the configuration context
    +     * @throws InitializationException if unable to connect to the Pulsar Broker
    +     * @throws UnsupportedAuthenticationException if the Broker URL uses a non-supported authentication mechanism
    +     */
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) throws InitializationException, UnsupportedAuthenticationException {
    +        createClient(context);
    +
    +        if (this.client == null) {
    +            throw new InitializationException("Unable to create Pulsar Client");
    +        }
    +    }
    +
    +    private void createClient(final ConfigurationContext context) throws InitializationException {
    +        try {
    +            this.client = getClientBuilder(context).build();
    +        } 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 ClientBuilder getClientBuilder(ConfigurationContext context) throws UnsupportedAuthenticationException {
    +        if (builder == null) {
    +           builder = PulsarClient.builder();
    +
    +           if (context.getProperty(ALLOW_TLS_INSECURE_CONNECTION).isSet()) {
    --- End diff --
    
    Are these properties always set, even in the case where someone goes in and changes the default value to a blank string?  That was the scenario I am trying to account for with these isSet() checks.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197452220
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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 org.apache.pulsar.client.api.Producer;
    +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.Mockito.mock;
    +
    +public class LRUCacheTest {
    +
    +    //@Mock
    --- End diff --
    
    Should remove unused code.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197458007
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    --- End diff --
    
    This is going to get called in the `@OnStopped` method when all threads have completed. I don't think we want to be calling it here.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r199829042
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    --- End diff --
    
    I am, seeing the following error in the "continuous-integration/appveyor/pr — AppVeyor " output:
    
    [ERROR] The build could not read 1 project -> [Help 1]
    1066[ERROR]   
    1067[ERROR]   The project org.apache.nifi:nifi-pulsar-bundle:1.7.0-SNAPSHOT (C:\projects\nifi\nifi-nar-bundles\nifi-pulsar-bundle\pom.xml) has 1 error
    1068[ERROR]     Non-resolvable parent POM: Could not find artifact org.apache.nifi:nifi-nar-bundles:pom:1.7.0-SNAPSHOT and 'parent.relativePath' points at wrong local POM @ line 19, column 13 -> [Help 2]
    
    Does this mean the parent POM changed?  Is there anything else you need from me to move this PR forward?


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197460284
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.pulsar.PulsarClientService;
    +import org.apache.pulsar.client.api.PulsarClient;
    +
    +public abstract class AbstractPulsarProcessor extends AbstractProcessor {
    +
    +    public 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(PulsarClientService.class)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles for which all content was sent to Pulsar.")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    --- End diff --
    
    The description here is tailored to the producer. We should avoid putting it into the abstract class.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197786403
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    +        }
    +    }
    +
    +    protected Consumer<T> getConsumer(ProcessContext context) throws PulsarClientException {
    +
    +        if (consumer == null) {
    +           ConsumerBuilder<T> builder = (ConsumerBuilder<T>) getPulsarClient(context).newConsumer();
    +
    +           if (context.getProperty(TOPICS).isSet()) {
    +              builder = builder.topic(context.getProperty(TOPICS)
    +                               .evaluateAttributeExpressions().getValue().split("[, ]"));
    --- End diff --
    
    @david-streamlio yes, that is true. However, it does not allow for commas separating with spaces. I.e., 90% of the time users will enter "myTopic, yourTopic" and that regex will split it into "myTopic" followed by " yourTopic".... note the space before the 'y' in the second topic name. It is best to trim these out. Otherwise, users end up getting back error messages like "Could not find topic  yourTopic" and then it's not clear that there's a leading space and a lot of time is spent trying to debug.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197452726
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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 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 List<PropertyDescriptor> properties;
    +    private volatile PulsarClient client;
    +    private boolean secure = false;
    +    private ClientBuilder builder;
    +
    +    static {
    +        final List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(PULSAR_SERVICE_URL);
    +        props.add(ALLOW_TLS_INSECURE_CONNECTION);
    +        props.add(CONCURRENT_LOOKUP_REQUESTS);
    +        props.add(CONNECTIONS_PER_BROKER);
    +        props.add(IO_THREADS);
    +        props.add(KEEP_ALIVE_INTERVAL);
    +        props.add(LISTENER_THREADS);
    +        props.add(MAXIMUM_LOOKUP_REQUESTS);
    +        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<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +       return properties;
    +    }
    +
    +    /**
    +     * @param context the configuration context
    +     * @throws InitializationException if unable to connect to the Pulsar Broker
    +     * @throws UnsupportedAuthenticationException if the Broker URL uses a non-supported authentication mechanism
    +     */
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) throws InitializationException, UnsupportedAuthenticationException {
    +        createClient(context);
    +
    +        if (this.client == null) {
    +            throw new InitializationException("Unable to create Pulsar Client");
    +        }
    +    }
    +
    +    private void createClient(final ConfigurationContext context) throws InitializationException {
    +        try {
    +            this.client = getClientBuilder(context).build();
    +        } 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 ClientBuilder getClientBuilder(ConfigurationContext context) throws UnsupportedAuthenticationException {
    +        if (builder == null) {
    +           builder = PulsarClient.builder();
    +
    +           if (context.getProperty(ALLOW_TLS_INSECURE_CONNECTION).isSet()) {
    --- End diff --
    
    Most of these properties has default values, so we can avoid checking if set. Any property that has a default value is always set.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197454197
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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"
    --- End diff --
    
    Can you explain what the trade-off here is? Does Async result in a larger chance of data duplication?


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    @MikeThomsen 
    
    I have updated my code to use the Apache Pulsar 2.0 client API, but cannot commit the changes yet, as the release vote is still pending, and thus the client jar file hasn't been released to the maven central repository.
    
    As for testing the processors within a docker environment, you can use the following steps that I have verified and written down here:
    
    Testing NiFi Processors
    
    1. Launch a docker container running Apache Pulsar 2.0:  
        docker pull apachepulsar/pulsar:2.0.0-rc1-incubating
        docker run -d -i --name pulsar -p 6650:6650 -p 8000:8000 apachepulsar/pulsar:2.0.0-rc1-incubating
    
    2. ssh into the pulsar container and start the Pulsar service
        docker exec -it pulsar /bin/bash
        root@266e559270ce:/pulsar/bin/pulsar standalone &
    
    Launch a docker container running apache NiFi 1.7.0-SNAPSHOT
          docker run -d -i --name nifi --link pulsar -p 8080:8080 apache/nifi:1.7.0-SNAPSHOT
    Load the following template to to NiFi, and start the ConsumePulsar processor FIRST.
    Then start the other processors.
    Verify that the data is being produced and consumed at the same rate.
    Open the Pulsar dashboard at http://localhost:8000 to see the topic was created and that the messages were generated.
    [Pulsar-Test-xml.txt](https://github.com/apache/nifi/files/2051691/Pulsar-Test-xml.txt)



---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197449111
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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)
    --- End diff --
    
    Rather than using a Boolean Validator it's probably best to use a set of Allowable Values: `.allowableValues("true", "false")`


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197460222
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.pulsar.PulsarClientService;
    +import org.apache.pulsar.client.api.PulsarClient;
    +
    +public abstract class AbstractPulsarProcessor extends AbstractProcessor {
    +
    +    public 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(PulsarClientService.class)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    --- End diff --
    
    The description here is tailored to the producer. We should avoid putting it into the abstract class.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    > Also, something to think about is whether Pulsar's client will work well across different broker versions. For example, when Pulsar 2.x comes out, will the 1.x client work well against a 2.x broker? or vice versa?
    
    Pulsar API compatibility vs binary protocol compatibility guidelines:
    * Protocol compatibility will be always ensured (unless major reasons)
    * API can be broken across major releases
    The problem is that Kafka broke protocol compatibility at every release, which necessitated the 0.9, .0.10, 0.11, and 1.0 versions.
    
    FYI, the 2.0 release is under voting, and should be officially available in maven central next week, but I have downloaded the code an built the jar locally and am in the process of replacing the deprecated classes with 2.0+ API changes.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197452243
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/test/java/org/apache/nifi/pulsar/cache/LRUCacheTest.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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 org.apache.pulsar.client.api.Producer;
    +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.Mockito.mock;
    +
    +public class LRUCacheTest {
    +
    +    //@Mock
    +    //private ResourcePool<PulsarProducer> mockedResourcePool;
    +
    +    @Mock
    +    private Producer mockedPulsarProducer;
    +
    +    @SuppressWarnings("unchecked")
    +    @Before
    +    public void setUp() throws InterruptedException {
    +     // mockedResourcePool = mock(ResourcePool.class);
    --- End diff --
    
    Should remove unused code.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197465549
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    --- End diff --
    
    Why are we creating a new FlowFile and sending it to failure? In this case, the original incoming FlowFile should go to failure, not a newly-created FlowFile.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197439995
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    --- End diff --
    
    Typo: "conneciton" should be "connection". Same for displayName.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197787589
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/pubsub/PublishPulsar.java ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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.pubsub;
    +
    +import java.io.ByteArrayOutputStream;
    +
    +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.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.exception.ProcessException;
    +import org.apache.nifi.processors.pulsar.AbstractPulsarProducerProcessor;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.pulsar.client.api.MessageId;
    +import org.apache.pulsar.client.api.Producer;
    +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.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.")
    +@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 AbstractPulsarProducerProcessor<byte[]> {
    +
    +    @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
    +        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.putAttribute(flowFile, "msg.count", "0");
    +            session.transfer(flowFile, REL_SUCCESS);
    +            return;
    +        }
    +
    +        try {
    +            Producer<byte[]> producer = getProducer(context, topic);
    +
    +            if (context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +                sendAsync(producer, session, messageContent);
    +                session.remove(flowFile);
    --- End diff --
    
    You should not be transferring the FlowFile to 'success' until you've received confirmation for all published message. If you do, you'll have a situation where the user has an incoming FlowFile, it fails to go to Pulsar, but still goes to the 'success' relationship.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    From the errors I am seeing in the CI log, it appears that this PR is being built against the 1.8.0-SNAPSHOT release? Is that correct?  Should I change the version in all of my POMs?
    
    [WARNING] 'dependencies.dependency.(groupId:artifactId:type:classifier)' must be unique: org.apache.nifi:nifi-utils:jar -> duplicate declaration of version 1.8.0-SNAPSHOT @ org.apache.nifi:nifi-couchbase-processors:[unknown-version], /home/travis/build/apache/nifi/nifi-nar-bundles/nifi-couchbase-bundle/nifi-couchbase-processors/pom.xml, line 65, column 21
     @ 
    [ERROR] The build could not read 1 project -> [Help 1]
    [ERROR]   
    [ERROR]   The project org.apache.nifi:nifi-pulsar-bundle:1.7.0-SNAPSHOT (/home/travis/build/apache/nifi/nifi-nar-bundles/nifi-pulsar-bundle/pom.xml) has 1 error
    [ERROR]     Non-resolvable parent POM for org.apache.nifi:nifi-pulsar-bundle:1.7.0-SNAPSHOT: Could not find artifact org.apache.nifi:nifi-nar-bundles:pom:1.7.0-SNAPSHOT in sonatype-snapshots (https://oss.sonatype.org/content/repositories/snapshots/) and 'parent.relativePath' points at wrong local POM @ line 19, column 13 -> [Help 2]


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197460635
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.pulsar.PulsarClientService;
    +import org.apache.pulsar.client.api.PulsarClient;
    +
    +public abstract class AbstractPulsarProcessor extends AbstractProcessor {
    +
    +    public 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(PulsarClientService.class)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles for which all content was sent to Pulsar.")
    +            .build();
    +
    +    public 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 PulsarClient client;
    +
    +    protected PulsarClient getPulsarClient(ProcessContext context) {
    +        if (client == null) {
    --- End diff --
    
    This is not thread-safe. All processors must be thread-safe.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197788509
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    --- End diff --
    
    You'll need to avoid committing the session until you've received an ACK/NACK for each message in that session. As-is, this is not transferring out the failed messages. What it is doing is creating a new, empty FlowFile that is not associated with the incoming FlowFile, and sending that empty FlowFile to failure.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    Nothing left for you at the moment. I keep getting side tracked with real work requirements. It's a big commit, so we'll need a lot of review.
    
    @alopresto @markap14 @bbende @ijokarumawak @joewitt  once 1.7 is out the door I can try to find some time, but I'd like others to look at this because it's a non-trivial commit that brings in a big chunk of totally new functionality.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r188922865
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml ---
    @@ -0,0 +1,40 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  Licensed to the Apache Software Foundation (ASF) under one or more
    +  contributor license agreements. See the NOTICE file distributed with
    +  this work for additional information regarding copyright ownership.
    +  The ASF licenses this file to You under the Apache License, Version 2.0
    +  (the "License"); you may not use this file except in compliance with
    +  the License. You may obtain a copy of the License at
    +  http://www.apache.org/licenses/LICENSE-2.0
    +  Unless required by applicable law or agreed to in writing, software
    +  distributed under the License is distributed on an "AS IS" BASIS,
    +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +  See the License for the specific language governing permissions and
    +  limitations under the License.
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <modelVersion>4.0.0</modelVersion>
    +
    +    <parent>
    +        <groupId>org.apache.nifi</groupId>
    +        <artifactId>nifi-pulsar-bundle</artifactId>
    +        <version>1.7.0-SNAPSHOT</version>
    +    </parent>
    +
    +    <artifactId>nifi-pulsar-client-service-api</artifactId>
    +    <packaging>jar</packaging>
    +
    +    <dependencies>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-api</artifactId>
    +            <scope>provided</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.pulsar</groupId>
    +            <artifactId>pulsar-client</artifactId>
    +            <version>1.21.0-incubating</version>
    --- End diff --
    
    1.22.0-incubating is the latest build. Might as well bump it up since there's about 3-4 months between those releases.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197445797
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)")
    --- End diff --
    
    Typo: "maked" should be "marked"


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by jzonthemtn <gi...@git.apache.org>.
Github user jzonthemtn commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r201497277
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/pom.xml ---
    @@ -0,0 +1,40 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  Licensed to the Apache Software Foundation (ASF) under one or more
    +  contributor license agreements. See the NOTICE file distributed with
    +  this work for additional information regarding copyright ownership.
    +  The ASF licenses this file to You under the Apache License, Version 2.0
    +  (the "License"); you may not use this file except in compliance with
    +  the License. You may obtain a copy of the License at
    +  http://www.apache.org/licenses/LICENSE-2.0
    +  Unless required by applicable law or agreed to in writing, software
    +  distributed under the License is distributed on an "AS IS" BASIS,
    +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +  See the License for the specific language governing permissions and
    +  limitations under the License.
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <modelVersion>4.0.0</modelVersion>
    +
    +    <parent>
    +        <groupId>org.apache.nifi</groupId>
    +        <artifactId>nifi-pulsar-bundle</artifactId>
    +        <version>1.7.0-SNAPSHOT</version>
    +    </parent>
    +
    +    <artifactId>nifi-pulsar-client-service-api</artifactId>
    +    <packaging>jar</packaging>
    +
    +    <dependencies>
    +        <dependency>
    +            <groupId>org.apache.nifi</groupId>
    +            <artifactId>nifi-api</artifactId>
    +            <scope>provided</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.pulsar</groupId>
    +            <artifactId>pulsar-client</artifactId>
    +            <version>2.0.0-rc1-incubating</version>
    --- End diff --
    
    I think now there is a `2.0.1-incubating` version..?


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    Does that mean this commit won't make the 1.7 release? 


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197459198
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    +        }
    +    }
    +
    +    protected Consumer<T> getConsumer(ProcessContext context) throws PulsarClientException {
    +
    +        if (consumer == null) {
    +           ConsumerBuilder<T> builder = (ConsumerBuilder<T>) getPulsarClient(context).newConsumer();
    +
    +           if (context.getProperty(TOPICS).isSet()) {
    +              builder = builder.topic(context.getProperty(TOPICS)
    +                               .evaluateAttributeExpressions().getValue().split("[, ]"));
    +           } else if (context.getProperty(TOPICS_PATTERN).isSet()) {
    --- End diff --
    
    If we are explicitly ignoring this property when "TOPICS" is set, then we should probably have a `customValidate` property that does not allow both to be set. Or, at a very minimum, document in the description of the "Topics Pattern" property that the Topics Pattern property is ignored if the Topic Names are set.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197462050
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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 "
    --- End diff --
    
    Avoid explicit time units


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197455120
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .defaultValue("5")
    +            .build();
    +
    +    public static final PropertyDescriptor RECEIVER_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Consumer receive queue size.")
    --- End diff --
    
    Typical naming convention for properties is Title Case and should avoid the "." at the end.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197457772
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    --- End diff --
    
    Is there any reason to wait here? We should probably just shutdown the pool and move on.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    @joewitt @markap14 @bbende Can one of you skim through @david-streamlio's use of controller services and let me know if you think he should refactor the names? My gut feeling is that he can rename the processors to very generic ones and have _1_X suffixed controller services so that if/when Pulsar goes 2.X the logic just gets updated in new NARs for that. To me, it looks like a lot of the work that is likely to change is in the controller services.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197449704
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    --- End diff --
    
    This property does not appear to be included in the List of Properties made available by the service. How does this relate to the SSL Context? Should this be removed?


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197577680
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    --- End diff --
    
    We are waiting for the asynchronous message acknowledgments to return in order to prevent duplicate data being sent to Pulsar in the event of a shutdown and restart of the processor


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197460837
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProcessor.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.pulsar.PulsarClientService;
    +import org.apache.pulsar.client.api.PulsarClient;
    +
    +public abstract class AbstractPulsarProcessor extends AbstractProcessor {
    --- End diff --
    
    This class serves simply to define a Property Descriptor and a 5-line convenience method. I would recommend removing the abstract class all together, as I believe that it would keep the code more readable & maintainable.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197578241
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPIC);
    +        properties.add(ASYNC_ENABLED);
    +        properties.add(MAX_ASYNC_REQUESTS);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    protected LRUCache<String, Producer<T>> producers;
    +
    +    // Pool for running multiple publish Async requests
    +    protected ExecutorService publisherPool;
    +    protected ExecutorCompletionService<Object> publisherService;
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            publisherPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +            publisherService = new ExecutorCompletionService<>(publisherPool);
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           // Stop all the async publishers
    +           try {
    +              publisherPool.shutdown();
    +              publisherPool.awaitTermination(20, TimeUnit.SECONDS);
    +           } catch (InterruptedException e) {
    +              getLogger().error("Unable to stop all the Pulsar Producers", e);
    +           }
    +       }
    +    }
    +
    +    @OnStopped
    +    public void cleanUp(final ProcessContext context) {
    +       getProducerCache(context).clear();
    +    }
    +
    +    @SuppressWarnings("rawtypes")
    +    protected void sendAsync(Producer producer, final ProcessSession session, final byte[] messageContent) {
    +        try {
    +            publisherService.submit(new Callable<Object>() {
    +               @Override
    +               public Object call() throws Exception {
    +                 try {
    +                     return producer.newMessage().value(messageContent).sendAsync().handle((msgId, ex) -> {
    +                        if (msgId != null) {
    +                           return msgId;
    +                        } else {
    +                           FlowFile flowFile = session.create();
    --- End diff --
    
    When I tried to use the original FlowFile, I would get runtime exceptions about the FlowFile NOT being successfully transferred to a relationship. Since this code is inside an async callback method, by the time it is invoked the original session has been closed. Therefore, my only option to transfer out the failed messages is to create a new FlowFile specifically for failures.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197458424
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    --- End diff --
    
    Typo: "aany" should be "any". At this point, we probably want to call `context.yield()` as well because we're just going to keep hitting this condition, I believe.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    I am going to close this PR and create a new one based on a fresh fork of the project


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197449565
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("30")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor MAXIMUM_LOOKUP_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum lookup requests")
    +            .description("Number of max lookup-requests allowed on each broker-connection to prevent overload on broker."
    +                    + "(default: 50000) It should be bigger than maxConcurrentLookupRequests. ")
    +            .required(false)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("50000")
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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 TLS_TRUST_CERTS_FILE_PATH = new PropertyDescriptor.Builder()
    +            .name("TLS Trust Certs File Path")
    +            .description("Set the path to the trusted TLS certificate file")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor USE_TCP_NO_DELAY = new PropertyDescriptor.Builder()
    +            .name("Use TCP no-delay 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 SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    --- End diff --
    
    This property does not appear to be included in the List of Properties made available by the service.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197452174
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java ---
    @@ -0,0 +1,59 @@
    +/*
    + * 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;
    +
    +public class LRUCache<K, V> {
    +
    +    private LinkedHashMap<K, V> lruCacheMap;
    +    private final int capacity;
    +    private final boolean SORT_BY_ACCESS = true;
    +    private final float LOAD_FACTOR = 0.75F;
    +    //private final ResourcePool<V> resourcePool;
    +
    +    public LRUCache(int capacity){
    +        this.capacity = capacity;
    +        this.lruCacheMap = new LinkedHashMap<>(capacity, LOAD_FACTOR, SORT_BY_ACCESS);
    +    }
    +
    +    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();
    +            lruCacheMap.remove(victimKey);
    +        }
    +        lruCacheMap.put(k, v);
    +    }
    +
    +    public int getSize() {
    +       return lruCacheMap.size();
    +    }
    +
    +    public void clear() {
    +        lruCacheMap.clear();
    +    }
    +
    +    public void printSequence() {
    +       System.out.println(lruCacheMap.keySet());
    --- End diff --
    
    Should avoid writing to StdOut. Can use Logger instead.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    @MikeThomsen I have updated the code to use the new Apache Pulsar 2.0 client API, but am blocked on the Apache release vote before the the client jar is published to the Maven central repo.
    
    In the meantime, I am having issues seeing the new Pulsar processors on the NiFi canvas using the docker image provided in the repo.  I have confirmed that the docker container has the nifi-pulsar-nar-1.7.0-SNAPSHOT.nar file in /opt/nifi/nifi-1.7.0-SNAPSHOT/lib/ directory, and have started NiFi, but when I a attempt to add a Pulsar Processor to the Canvas, none of them are available.
    
    I have double checked the class names in the META-INF/services/org.apache.nifi.processor.Processor file and confirmed they are correct. I have also confirmed that the classes exist in the nifi-pulsar-nar-1.7.0-SNAPSHOT.nar file.   Any ideas on what might be missing / wrong?
    
    ![screen shot 2018-05-29 at 10 27 27 am](https://user-images.githubusercontent.com/35466513/40674893-ef1f16a4-632a-11e8-8f81-e30386969091.png)



---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197459885
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    +        }
    +    }
    +
    +    protected Consumer<T> getConsumer(ProcessContext context) throws PulsarClientException {
    +
    +        if (consumer == null) {
    +           ConsumerBuilder<T> builder = (ConsumerBuilder<T>) getPulsarClient(context).newConsumer();
    +
    +           if (context.getProperty(TOPICS).isSet()) {
    +              builder = builder.topic(context.getProperty(TOPICS)
    +                               .evaluateAttributeExpressions().getValue().split("[, ]"));
    +           } else if (context.getProperty(TOPICS_PATTERN).isSet()) {
    +              builder = builder.topicsPattern(context.getProperty(TOPICS_PATTERN).getValue());
    +           } else {
    +             throw new PulsarClientException("No topic specified.");
    +           }
    +
    +           if (context.getProperty(SUBSCRIPTION_NAME).isSet()) {
    +             builder = builder.subscriptionName(context.getProperty(SUBSCRIPTION_NAME).getValue());
    +           } else {
    +             throw new PulsarClientException("No subscription specified.");
    +           }
    +
    +           if (context.getProperty(ACK_TIMEOUT).isSet()) {
    --- End diff --
    
    No need to check that property is set for any property that either has a default value or is required.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r188938042
  
    --- Diff: 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<R extends PoolableResource> implements ResourcePool<R> {
    +
    +    private final Lock lock = new ReentrantLock();
    +    private final Condition poolAvailable = lock.newCondition();
    +    private int max_resources;
    +    private final Vector<R> pool;
    +
    +    private final ResourceExceptionHandler resourceExceptionHandler;
    +    private final ResourceFactory<R> factory;
    +
    +    public ResourcePoolImpl(ResourceFactory<R> factory, int max_resources) {
    +            this(factory, new ResourceExceptionHandlerImpl(), max_resources);
    +    }
    +
    +    public ResourcePoolImpl(ResourceFactory<R> factory, ResourceExceptionHandler handler, int max_resources) {
    +        lock.lock();
    +        try {
    +            this.factory = factory;
    +            this.resourceExceptionHandler = handler;
    +            this.max_resources = max_resources;
    +            this.pool = new Vector<R>(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() {
    +
    --- End diff --
    
    There's a lot of extraneous white space in some of these methods. It would be nice if you could skim through and clean some of that up. Not a deal breaker, but makes things tighter.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197454275
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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)
    --- End diff --
    
    We should user `allowableValues("true", "false")` instead of a boolean validator.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197454564
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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 "
    --- End diff --
    
    Want to avoid explicit time period and instead let the user enter time duration as desired.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197461736
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarProducerProcessor.java ---
    @@ -0,0 +1,323 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.pulsar.cache.LRUCache;
    +import org.apache.pulsar.client.api.CompressionType;
    +import org.apache.pulsar.client.api.MessageRoutingMode;
    +import org.apache.pulsar.client.api.Producer;
    +import org.apache.pulsar.client.api.ProducerBuilder;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +
    +public abstract class AbstractPulsarProducerProcessor<T> extends AbstractPulsarProcessor {
    +
    +    public static final String MSG_COUNT = "msg.count";
    +    public static final String TOPIC_NAME = "topic.name";
    +
    +    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(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .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 MAX_ASYNC_REQUESTS = new PropertyDescriptor.Builder()
    +            .name("Maximum Async Requests")
    +            .description("The maximum number of outstanding asynchronous publish 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 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)
    --- End diff --
    
    Should use allowable values


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197452036
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service-api/src/main/java/org/apache/nifi/pulsar/cache/LRUCache.java ---
    @@ -0,0 +1,59 @@
    +/*
    + * 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;
    +
    +public class LRUCache<K, V> {
    +
    +    private LinkedHashMap<K, V> lruCacheMap;
    +    private final int capacity;
    +    private final boolean SORT_BY_ACCESS = true;
    +    private final float LOAD_FACTOR = 0.75F;
    +    //private final ResourcePool<V> resourcePool;
    --- End diff --
    
    Should remove unused code.


---

[GitHub] nifi issue #2702: Added Apache Pulsar processors

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on the issue:

    https://github.com/apache/nifi/pull/2702
  
    Yes, you should rebase against `master` as it appears the 1.7.0 release occurred after this PR was open, and current master is 1.8.0-SNAPSHOT. 


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by david-streamlio <gi...@git.apache.org>.
Github user david-streamlio commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197619121
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    +        }
    +    }
    +
    +    protected Consumer<T> getConsumer(ProcessContext context) throws PulsarClientException {
    +
    +        if (consumer == null) {
    +           ConsumerBuilder<T> builder = (ConsumerBuilder<T>) getPulsarClient(context).newConsumer();
    +
    +           if (context.getProperty(TOPICS).isSet()) {
    +              builder = builder.topic(context.getProperty(TOPICS)
    +                               .evaluateAttributeExpressions().getValue().split("[, ]"));
    --- End diff --
    
    The call to split("[, ]") interprets its parameter as a regex, so the pattern '[, ]' means split on either a comma or a space... This allows the user to use a CSV list or a space-separated list. https://docs.oracle.com/javase/8/docs/api/java/lang/String.html#split-java.lang.String-


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197458789
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    +        }
    +    }
    +
    +    protected Consumer<T> getConsumer(ProcessContext context) throws PulsarClientException {
    +
    +        if (consumer == null) {
    +           ConsumerBuilder<T> builder = (ConsumerBuilder<T>) getPulsarClient(context).newConsumer();
    +
    +           if (context.getProperty(TOPICS).isSet()) {
    +              builder = builder.topic(context.getProperty(TOPICS)
    +                               .evaluateAttributeExpressions().getValue().split("[, ]"));
    --- End diff --
    
    I don't think we want to be splitting on explicit white-space here. Probably should instead split on "," and then `trim()` all values.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197445351
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue("1")
    +            .build();
    +
    +    public static final PropertyDescriptor KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive interval")
    +            .displayName("Keep Alive interval")
    +            .description("The keep alive interval in seconds for each client-broker-connection. (default: 30).")
    --- End diff --
    
    We don't want to specify a time unit in description and accept a number. Instead, we should use `StandardValidators.TIME_PERIOD_VALIDATOR` and allow the user to enter any time unit that they want. You can then default to "30 secs". We can then get the number of seconds via `context.getProperty(KEEP_ALIVE_INTERVAL).asTimePeriod(TimeUnit.SECONDS);`


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197444911
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-client-service/src/main/java/org/apache/nifi/pulsar/StandardPulsarClientService.java ---
    @@ -0,0 +1,308 @@
    +/*
    + * 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.lifecycle.OnEnabled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.pulsar.client.api.ClientBuilder;
    +import org.apache.pulsar.client.api.PulsarClient;
    +import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
    +import org.apache.pulsar.client.impl.auth.AuthenticationTls;
    +
    +public class StandardPulsarClientService extends AbstractControllerService implements PulsarClientService {
    +
    +    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)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +
    +    public static final PropertyDescriptor ALLOW_TLS_INSECURE_CONNECTION = new PropertyDescriptor.Builder()
    +            .name("Allow TLS insecure conneciton")
    +            .displayName("Allow TLS insecure conneciton")
    +            .description("Whether the Pulsar client will accept untrusted TLS certificate from broker or not.")
    +            .required(false)
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .defaultValue(Boolean.FALSE.toString())
    +            .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 "
    --- End diff --
    
    We want to avoid specifying default values in the property's description (this is done for most properties here, it seems). The documentation will already render the default value based on what is set by the `defaultValue` method. If the value is ever changed, it's easy to forget to update the description, and this leads to confusion and incorrectness in the documentation, plus is just more to maintain.


---

[GitHub] nifi pull request #2702: Added Apache Pulsar processors

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2702#discussion_r197459415
  
    --- Diff: nifi-nar-bundles/nifi-pulsar-bundle/nifi-pulsar-processors/src/main/java/org/apache/nifi/processors/pulsar/AbstractPulsarConsumerProcessor.java ---
    @@ -0,0 +1,296 @@
    +/*
    + * 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.Set;
    +import java.util.concurrent.ExecutorCompletionService;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.RejectedExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +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.expression.ExpressionLanguageScope;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.pulsar.client.api.Consumer;
    +import org.apache.pulsar.client.api.ConsumerBuilder;
    +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction;
    +import org.apache.pulsar.client.api.Message;
    +import org.apache.pulsar.client.api.PulsarClientException;
    +import org.apache.pulsar.client.api.SubscriptionType;
    +
    +public abstract class AbstractPulsarConsumerProcessor<T> 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");
    +
    +    static final AllowableValue CONSUME = new AllowableValue(ConsumerCryptoFailureAction.CONSUME.name(), "Consume",
    +            "Mark the message as consumed despite being unable to decrypt the contents");
    +    static final AllowableValue DISCARD = new AllowableValue(ConsumerCryptoFailureAction.DISCARD.name(), "Discard",
    +            "Discard the message and don't perform any addtional processing on the message");
    +    static final AllowableValue FAIL = new AllowableValue(ConsumerCryptoFailureAction.FAIL.name(), "Fail",
    +            "Report a failure condition, and the route the message contents to the FAILED relationship.");
    +
    +    public static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
    +            .name("topics")
    +            .displayName("Topic Names")
    +            .description("Specify the topics this consumer will subscribe on. "
    +                    + "You can specify multiple topics in a comma-separated list."
    +                    + "E.g topicA, topicB, topicC ")
    +            .required(false)
    +            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TOPICS_PATTERN = new PropertyDescriptor.Builder()
    +            .name("Topics Pattern")
    +            .description("Specify a pattern for topics that this consumer will subscribe on. "
    +                    + "It accepts regular expression and will be compiled into a pattern internally. "
    +                    + "E.g. \"persistent://prop/use/ns-abc/pattern-topic-.*\"")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SUBSCRIPTION_NAME = new PropertyDescriptor.Builder()
    +            .name("Subscription Name")
    +            .description("Specify the subscription name for this consumer.")
    +            .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. A value of 0 means there is no timeout. If a non-zero value "
    +                    + "is sepcified, then messages that are not acknowledged within the configured"
    +                    + " timeout will be replayed.")
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSUMER_NAME = new PropertyDescriptor.Builder()
    +            .name("Consumer Name")
    +            .description("Set the name of the consumer to uniquely identify this client on the Broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .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.NON_NEGATIVE_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();
    +
    +    protected static final List<PropertyDescriptor> PROPERTIES;
    +    protected static final Set<Relationship> RELATIONSHIPS;
    +
    +    static {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(PULSAR_CLIENT_SERVICE);
    +        properties.add(TOPICS);
    +        properties.add(TOPICS_PATTERN);
    +        properties.add(SUBSCRIPTION_NAME);
    +        properties.add(CONSUMER_NAME);
    +        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<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    protected Consumer<T> consumer;
    +    protected ExecutorService consumerPool;
    +    protected ExecutorCompletionService<Message<T>> consumerService;
    +    protected ExecutorService ackPool;
    +    protected ExecutorCompletionService<Object> ackService;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void init(ProcessContext context) {
    +       if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +           consumerPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger());
    +           consumerService = new ExecutorCompletionService<>(consumerPool);
    +           ackPool = Executors.newFixedThreadPool(context.getProperty(MAX_ASYNC_REQUESTS).asInteger() + 1);
    +           ackService = new ExecutorCompletionService<>(ackPool);
    +       }
    +    }
    +
    +    @OnUnscheduled
    +    public void shutDown(final ProcessContext context) {
    +        if (context.getProperty(ASYNC_ENABLED).isSet() && context.getProperty(ASYNC_ENABLED).asBoolean()) {
    +            try {
    +                consumerPool.shutdown();
    +                consumerPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to stop all the Pulsar Consumers", e);
    +            }
    +
    +            try {
    +                ackPool.shutdown();
    +                ackPool.awaitTermination(10, TimeUnit.SECONDS);
    +            } catch (InterruptedException e) {
    +                getLogger().error("Unable to wait for all of the message acknowledgments to be sent", e);
    +            }
    +        }
    +        close(context);
    +    }
    +
    +    @OnStopped
    +    public void close(final ProcessContext context) {
    +        try {
    +            getLogger().info("Disconnecting Pulsar Consumer");
    +            if (consumer != null) {
    +              consumer.close();
    +            }
    +        } catch (Exception e) {
    +           getLogger().error("Unable to close Pulsar consumer", e);
    +        } finally {
    +           consumer = null;
    +        }
    +    }
    +
    +    protected void consumeAsync(ProcessContext context, ProcessSession session) throws PulsarClientException {
    +        Consumer<T> consumer = getConsumer(context);
    +
    +        try {
    +            consumerService.submit(() -> {
    +               return consumer.receive();
    +            });
    +        } catch (final RejectedExecutionException ex) {
    +            getLogger().error("Unable to consume aany more Pulsar messages", ex);
    +        }
    +    }
    +
    +    protected Consumer<T> getConsumer(ProcessContext context) throws PulsarClientException {
    +
    +        if (consumer == null) {
    +           ConsumerBuilder<T> builder = (ConsumerBuilder<T>) getPulsarClient(context).newConsumer();
    +
    +           if (context.getProperty(TOPICS).isSet()) {
    +              builder = builder.topic(context.getProperty(TOPICS)
    +                               .evaluateAttributeExpressions().getValue().split("[, ]"));
    +           } else if (context.getProperty(TOPICS_PATTERN).isSet()) {
    +              builder = builder.topicsPattern(context.getProperty(TOPICS_PATTERN).getValue());
    +           } else {
    +             throw new PulsarClientException("No topic specified.");
    --- End diff --
    
    We should have `customValidate` check this, rather than throwing an Exception when trying to create the consumer. This would prevent us from ever getting to this point.


---