You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by JPercivall <gi...@git.apache.org> on 2016/04/29 00:13:06 UTC

[GitHub] nifi pull request: Nifi 1808

GitHub user JPercivall opened a pull request:

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

    Nifi 1808

    Using @r-b-us original PR[1] as a base, I refactored it a bit and added more functionality. 
    
    [1] https://github.com/apache/nifi/pull/381

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

    $ git pull https://github.com/JPercivall/nifi NIFI-1808

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

    https://github.com/apache/nifi/pull/392.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 #392
    
----
commit 8914fe3cd93d09c6a1dfe538a07091b3cfe24c4a
Author: info@richards-tech.com <in...@richards-tech.com>
Date:   2016-04-26T15:35:54Z

    Added mqtt
    
    Fixed attribute name in PutMQTT

commit 8e125fdb73d0318e8772b822da2d34b1c1a08f34
Author: jpercivall <jo...@yahoo.com>
Date:   2016-04-28T21:53:40Z

    NIFI-1808 initial refactoring of general MQTT processors

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63581403
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    --- End diff --
    
    ? This is doing much more validation than just checking if it's blank. It verifies the URI, an empty path and that it's scheme is either tcp or ssl.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63592512
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    --- End diff --
    
    True but it should be clear for non-native speakers as well, so your point of view is just as (if not more) valid :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63581819
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    --- End diff --
    
    As a whole yes, I was pointing to the line above and if you can use an existing validator for it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63536744
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    --- End diff --
    
    Ah, that is a good point. I will add the check(s). Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r61949840
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/SubscribeMQTT.java ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT"})
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({@WritesAttribute(attribute="broker", description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute="topic", description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute="qos", description="The quality of service for this message."),
    +    @WritesAttribute(attribute="isDuplicate", description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute="isRetained", description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published on the topic.")})
    +public class SubscribeMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .allowableValues("0", "1", "2")
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private String broker;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +    private ScheduledExecutorService autoReconnectExecutor;
    +    private final Object autoReconnectLock = new Object();
    +    private ScheduledFuture<?> autoReconnectScheduledFuture;
    +    private static int AUTO_RECONNECT_PERIOD = 5;
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +
    +    LinkedBlockingQueue<MQTTQueueMessage> mqttQueue = new LinkedBlockingQueue<>();
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +     private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC_FILTER);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_MAX_QUEUE_SIZE);
    +
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_MESSAGE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            String topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new SubscribeMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +                mqttClient.subscribe(topicFilter, context.getProperty(PROP_QOS).asInteger());
    +            }
    +            autoReconnectExecutor = new ScheduledThreadPoolExecutor(1);
    +        } catch(MqttException me) {
    +            getLogger().error("msg "+me.getMessage());
    +        }
    +    }
    --- End diff --
    
    I agree, I will refactor it a bit


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63532252
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    --- End diff --
    
    Not sure this whole thing is valid since  @OnUnscheduled is invoked before @OnStopped which means the _onTrigger(..)_ can still be in progress where you are doing _reconnect(..)_ essentially creating race condition which could have undesirable side-effects. For example, assume _reconnect(..)_ succeeded releasing the lock and you are going into _ transferQueue(..)_, but right before it the @OnUnscheduled has disconnected.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on the pull request:

    https://github.com/apache/nifi/pull/392#issuecomment-221343071
  
    +1, all is good, test are passing.
    You'll need another PR for 0.x branch since the commits can not be cherry picked cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r61954882
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    --- End diff --
    
    The framework is designed to guarantee processor with an input (and no ScheduleWhenEmpty annotation) will only be triggered when it has a FlowFile to process. The only time I know of where a processor will spin is the case described in [NIFI-53](https://issues.apache.org/jira/browse/NIFI-53). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63583614
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    --- End diff --
    
    You're suggesting to use a validator to check if the URI path is blank?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61913088
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    --- End diff --
    
    So, here we are back into the discussion as to wether the pattern of obtaining connection in @OnSchedule is the right one. The history shows that it is not and in fact in Kafka we are not only doing it in _onTrigger()_, but also support re-initializing the resource in _onTrigger()_. Perhaps something to think about. Here is the link https://github.com/apache/nifi/pull/366/files#diff-da2dce9eaa10794cc440b30e94d4d5ccR169


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r61949787
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/SubscribeMQTT.java ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT"})
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({@WritesAttribute(attribute="broker", description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute="topic", description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute="qos", description="The quality of service for this message."),
    +    @WritesAttribute(attribute="isDuplicate", description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute="isRetained", description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published on the topic.")})
    +public class SubscribeMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .allowableValues("0", "1", "2")
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private String broker;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +    private ScheduledExecutorService autoReconnectExecutor;
    +    private final Object autoReconnectLock = new Object();
    +    private ScheduledFuture<?> autoReconnectScheduledFuture;
    +    private static int AUTO_RECONNECT_PERIOD = 5;
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +
    +    LinkedBlockingQueue<MQTTQueueMessage> mqttQueue = new LinkedBlockingQueue<>();
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +     private List<PropertyDescriptor> descriptors;
    --- End diff --
    
    Good catch, will fix


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63536287
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    Yup this is a conundrum in terms of usability. This exception will hit in the event the processor is scheduled to run at a specific time and it is stopped before the ontrigger occurs. So messages could have been received but the processor never ran.
    Upon seeing this, they have a couple options, start the processor again to continue processing the messages when the processor is triggered. To clear the internal queue they could re-create the processor (copy/paste and delete original). This kinda sucks from a usability stand point but to lose messages it should be pretty explicit. 
    Do you have another suggestion how to handle this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63536461
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    +            if (mqttQueue.size() >= maxQueueSize){
    +                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
    +            } else {
    +                mqttQueue.add(new MQTTQueueMessage(topic, message));
    +            }
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            logger.warn("Received MQTT 'delivery complete' message to subscriber:"+ token);
    +        }
    +    }
    +
    +    // Public for testing
    +    public void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new ConsumeMQTTCallback());
    +                mqttClient.connect(connOpts);
    +                if(!subscribed){
    +                    mqttClient.subscribe(topicFilter, qos);
    +                    subscribed = true;
    +                }
    +            }
    +        }
    +    }
    +
    +    // Public for testing
    +    public boolean isConnected(){
    --- End diff --
    
    Agrgh ..... Just not a fan of creating an operation specifically designed to simplify testing. Breaks the basic rules of encapsulation. I am ok with it being there if there are other uses for it, but as the comment currently states i don't think its right.
    Also, given what you are doing with 'mqttClient' the implementation is NPE waiting to be caught since _mqttClient_ can become _null_ right after _!null_ check succeeded (even thought you are not setting it to null anywhere at this point, but can easily be overlooked). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63907479
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QOS_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BROKER_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RETAIN_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BROKER_VALIDATOR)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder().name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Last Will Topic")
    +            .description("The topic to send the client's Last Will to. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_MESSAGE = new PropertyDescriptor.Builder()
    +            .name("Last Will Message")
    +            .description("The message to send as the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Last Will Retain")
    +            .description("Whether to retain the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .allowableValues("true","false")
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_QOS = new PropertyDescriptor.Builder()
    +            .name("Last Will QoS Level")
    +            .description("QoS level to be used when publishing the Last Will Message")
    +            .required(false)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2
    +            )
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CLEAN_SESSION = new PropertyDescriptor.Builder()
    +            .name("Session state")
    +            .description("Whether to start afresh or resume previous flows. See the allowable value descriptions for more details.")
    +            .required(true)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_TRUE,
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_FALSE
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_CLEAN_SESSION_TRUE.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MQTT_VERSION = new PropertyDescriptor.Builder()
    +            .name("MQTT Specification Version")
    +            .description("The MQTT specification version when connecting with the broker. See the allowable value descriptions for more details.")
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_MQTT_VERSION_AUTO,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_311,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_310
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_MQTT_VERSION_AUTO.getValue())
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CONN_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Connection Timeout (seconds)")
    +            .description("Maximum time interval the client will wait for the network connection to the MQTT server " +
    +                    "to be established. The default timeout is 30 seconds. " +
    +                    "A value of 0 disables timeout processing meaning the client will wait until the network connection is made successfully or fails.")
    +            .required(false)
    +            .defaultValue("30")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive Interval (seconds)")
    +            .description("Defines the maximum time interval between messages sent or received. It enables the " +
    +                    "client to detect if the server is no longer available, without having to wait for the TCP/IP timeout. " +
    +                    "The client will ensure that at least one message travels across the network within each keep alive period. In the absence of a data-related message during the time period, " +
    +                    "the client sends a very small \"ping\" message, which the server will acknowledge. A value of 0 disables keepalive processing in the client.")
    +            .required(false)
    +            .defaultValue("60")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static List<PropertyDescriptor> getAbstractPropertyDescriptors(){
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(PROP_BROKER_URI);
    +        descriptors.add(PROP_CLIENTID);
    +        descriptors.add(PROP_USERNAME);
    +        descriptors.add(PROP_PASSWORD);
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(PROP_LAST_WILL_TOPIC);
    +        descriptors.add(PROP_LAST_WILL_MESSAGE);
    +        descriptors.add(PROP_LAST_WILL_RETAIN);
    +        descriptors.add(PROP_LAST_WILL_QOS);
    +        descriptors.add(PROP_CLEAN_SESSION);
    +        descriptors.add(PROP_MQTT_VERSION);
    +        descriptors.add(PROP_CONN_TIMEOUT);
    +        descriptors.add(PROP_KEEP_ALIVE_INTERVAL);
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
    +        final List<ValidationResult> results = new ArrayList<>(1);
    +        final boolean usernameSet = validationContext.getProperty(PROP_USERNAME).isSet();
    +        final boolean passwordSet = validationContext.getProperty(PROP_PASSWORD).isSet();
    +
    +        if ((usernameSet && !passwordSet) || (!usernameSet && passwordSet)) {
    +            results.add(new ValidationResult.Builder().subject("Username and Password").valid(false).explanation("if username or password is set, both must be set").build());
    +        }
    +
    +        final boolean lastWillTopicSet = validationContext.getProperty(PROP_LAST_WILL_TOPIC).isSet();
    +        final boolean lastWillMessageSet = validationContext.getProperty(PROP_LAST_WILL_MESSAGE).isSet();
    +
    +        final boolean lastWillRetainSet = validationContext.getProperty(PROP_LAST_WILL_RETAIN).isSet();
    +        final boolean lastWillQosSet = validationContext.getProperty(PROP_LAST_WILL_QOS).isSet();
    +
    +        // If any of the Last Will Properties are set
    +        if(lastWillTopicSet || lastWillMessageSet || lastWillRetainSet || lastWillQosSet){
    +            // And any are not set
    +            if(!(lastWillTopicSet && lastWillMessageSet && lastWillRetainSet && lastWillQosSet)){
    +                // Then mark as invalid
    +                results.add(new ValidationResult.Builder().subject("Last Will Properties").valid(false).explanation("if any of the Last Will Properties (message, topic, retain and QoS) are " +
    +                        "set, all must be set.").build());
    +            }
    +        }
    +
    +        return results;
    +    }
    +
    +    public static Properties transformSSLContextService(SSLContextService sslContextService){
    +        Properties properties = new Properties();
    +        properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
    --- End diff --
    
    Also, in PropertyDescriptors consider mentioning which MQTT standard property a particular PropertyDescriptor corresponds to. Since we have strong believe that our names are more descriptive then standard properties we should at least help knowledgeable user to draw correlation between NiFi property and the actual MQTT property it corresponds to. I started doing it in new Kafka stuff as well, helps with referencing Kafka docs and see which exact property you may need more info on.
    Also, linking to some online doc where each of these property is described would be nice as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the pull request:

    https://github.com/apache/nifi/pull/392#issuecomment-219860709
  
    @olegz just pushed out a new patch addressing the second round of comments and rebased to master to fix the merge conflicts


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63918814
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QOS_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RETAIN_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are transferred to this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are transferred to this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    --- End diff --
    
    But didn't you say before that you don't really care about the connection since onTrigger operates on internal queue?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on the pull request:

    https://github.com/apache/nifi/pull/392#issuecomment-216610646
  
    Also, can you update the PR message with some description? Currently it only shows 'NIFI-1808'


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63578922
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    +
    +        try {
    +            /*
    +             * Underlying method waits for the message to publish (according to set QoS), so it executes synchronously:
    +             *     MqttClient.java:361 aClient.publish(topic, message, null, null).waitForCompletion(getTimeToWait());
    +             */
    +            mqttClient.publish(topic, mqttMessage);
    +            session.transfer(flowfile, REL_SUCCESS);
    +        } catch(MqttException me) {
    +            logger.error("Failed to publish message.", me);
    +            session.transfer(flowfile, REL_FAILURE);
    +        }
    +    }
    +
    +    private class PublishMQTTCallback  implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and re-connect failed");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.error("Message arrived to a PublishMQTT processor { topic:'" + topic +"; payload:"+ Arrays.toString(message.getPayload())+"}");
    --- End diff --
    
    ```logger.error```? Is that intentional or should it be ```info```?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63540545
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    +            if (mqttQueue.size() >= maxQueueSize){
    +                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
    +            } else {
    +                mqttQueue.add(new MQTTQueueMessage(topic, message));
    +            }
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            logger.warn("Received MQTT 'delivery complete' message to subscriber:"+ token);
    +        }
    +    }
    +
    +    // Public for testing
    +    public void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new ConsumeMQTTCallback());
    +                mqttClient.connect(connOpts);
    +                if(!subscribed){
    +                    mqttClient.subscribe(topicFilter, qos);
    +                    subscribed = true;
    +                }
    +            }
    +        }
    +    }
    +
    +    // Public for testing
    +    public boolean isConnected(){
    --- End diff --
    
    So, Reflection is a very common pattern to be used for such things. Basically, unless you have any reason other then testing to make it public its fine, but if its only testing then consider creating a reflection-based callback in some test utils. 
    Also, look at it from the different angle. The tighter the code base is the easier it will be to introduce changes to in the future. Making it public you are immediately adding it to any and all future discussions on breaking changes regardless if when/if they happen. 
    
    Yes, I do see that you are not setting it to null explicitly and for that part of the comment its OK, yet it still feels a bit odd, but I am ok for now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61916053
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/SubscribeMQTT.java ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT"})
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({@WritesAttribute(attribute="broker", description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute="topic", description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute="qos", description="The quality of service for this message."),
    +    @WritesAttribute(attribute="isDuplicate", description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute="isRetained", description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published on the topic.")})
    +public class SubscribeMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .allowableValues("0", "1", "2")
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private String broker;
    --- End diff --
    
    This is more of a general comment. Please review all of these attributes to see if any of them can be modified by a different threads. If so, then make them ```volatile```.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63584936
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    --- End diff --
    
    "are transferred to this relationship", sound more appropriate? ie:
    
    "FlowFiles that are sent successfully to the destination are transferred to relationship." 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63579631
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    --- End diff --
    
    I had this discussion with @joewitt on following certain conventions (see Kafka PR). Joe won, so now I am pushing it on you ;) We should follow convention where static final constants should all be uppercase (as property descriptors, relationships etc)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63908853
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QOS_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RETAIN_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are transferred to this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are transferred to this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    --- End diff --
    
    Assuming you are synchronizing for _reconnect_ purposes only, but _reconnect_ is already synchronized internally. Any reason for having nesting these synch blocks?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the pull request:

    https://github.com/apache/nifi/pull/392#issuecomment-220432007
  
    @olegz pushed out a commit addressing the comments (sans the SSL Context Service property mapping, due to my comment above)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63585043
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    --- End diff --
    
    Ok, if that's a convention then I am fine


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63585015
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RetainValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BrokerValidator)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    --- End diff --
    
    Yup that's a good point, I will double check


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63539712
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RetainValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BrokerValidator)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder().name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Last Will Topic")
    +            .description("The topic to send the client's Last Will to. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_MESSAGE = new PropertyDescriptor.Builder()
    +            .name("Last Will Message")
    +            .description("The message to send as the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Last Will Retain")
    +            .description("Whether to retain the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .allowableValues("true","false")
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_QOS = new PropertyDescriptor.Builder()
    +            .name("'Last Will' QoS Level")
    +            .description("QoS level to be used when publishing the Will Message")
    +            .required(false)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2
    +            )
    --- End diff --
    
    This was intentional, because some applications may not have a Last Will and there is no point forcing a use to have it set when it wouldn't be used. 
    That said I should have better validation to make sure it is set if the other Last Will properties are.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63900979
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private volatile long maxQueueSize;
    +
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +    private AtomicBoolean scheduled = new AtomicBoolean(false);
    --- End diff --
    
    minor: you can safely make it 'final'


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the pull request:

    https://github.com/apache/nifi/pull/392#issuecomment-219721262
  
    @olegz I pushed out new commits which address your comments. Also I added a bit more functionality I noticed other MQTT projects expose. Lastly I overhauled the unit tests to use a mock broker, marked the existing ones "integration" (since they actually created an MQTT broker that uses network ports) and excluded them from the maven build. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63533978
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    --- End diff --
    
    Don't feel bad about being picky when it come to concurrency. I will fix


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63918463
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QOS_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RETAIN_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are transferred to this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are transferred to this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    --- End diff --
    
    I agree that the synch block here should be removed since the reconnect already does it.
    
    I disagree that synchronization is only needed in the OnTrigger. The OnUnScheduled method will by definition have the potential to run while an OnTrigger is running so it will need the synchronization. Also there is no real detriment to doing synchronization in the other methods and will help cover the instances where unusual concurrency situations happen.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63919624
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QOS_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RETAIN_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are transferred to this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are transferred to this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    --- End diff --
    
    That is for ConsumeMQTT (and will attempt to reconnect if need be in the onTrigger), for PublishMQTT (that this comment is on) it pushes out messages using the connection in each onTrigger


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63577324
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    --- End diff --
    
    "sent out this relationship." - does it sound right? "sent out to. . ."? (in description of both relationships)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61913245
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    --- End diff --
    
    Also, although I am still reviewing I am assuming you have to obtain connection in both processors, which means some of this logic may be better in the abstract base class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63544945
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    "There still is the chance the processor would finish draining the queue" - indeed there is and that is why I pointed out to that abstract Kafka processor. I know it can't be applied one-for-one at its current state, but some aspects of it can. For example, it tracks how many tasks are currently executing. This accomplishes two things:
    1. The reconnect won't happen unless all tasks completed
    2. It has _acceptTasks_ flag to control the acceptance of new tasks to disallow new tasks until "some condition" resets the flag which allows for a clean resets of connection and other resources. This second point also brings an interesting point in your current code. If there are multiple tasks executing onTrigger and at some point one resets the connection (regardless of the reason) while other referenced previous instance of such connection?
    
    By knowing/controlling the above you would always know that there are live tasks which will essentially drain the queue and if there are not then you would drain it as part of the graceful stop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63539198
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    +            if (mqttQueue.size() >= maxQueueSize){
    +                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
    +            } else {
    +                mqttQueue.add(new MQTTQueueMessage(topic, message));
    +            }
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            logger.warn("Received MQTT 'delivery complete' message to subscriber:"+ token);
    +        }
    +    }
    +
    +    // Public for testing
    +    public void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new ConsumeMQTTCallback());
    +                mqttClient.connect(connOpts);
    +                if(!subscribed){
    +                    mqttClient.subscribe(topicFilter, qos);
    +                    subscribed = true;
    +                }
    +            }
    +        }
    +    }
    +
    +    // Public for testing
    +    public boolean isConnected(){
    --- End diff --
    
    It is used in the processor as well when checking in the OnTrigger (and should be used in the OnUnScheduled). The comment is just stating that it is "public" so that it can be used for testing. Is there another way you prefer exposing methods for testing?
    Is there another way you suggest checking for null or handling this situation? I'm never explicitly setting it to null (checked using intellij "Find Usages") so unless there is some extremely unfortunate timing I don't think it is worth worrying about.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63543066
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    This isn't exactly like Kafka because with Kafka you have to pull the records from Kafka. MQTT pushes the messages to you so you have to have an internal queue to buffer them. I'm looking at both AbstractKafkaProcessors in pr/366 but don't see how this applies.
    
    Why would the drain be initiated when the processor stopping (so started in the unscheduled)? There still is the chance the processor would finish draining the queue by itself in any remaining onTrigger calls. Also I'm already essentially doing an onTrigger in the stop. I'm using the same method that onTrigger uses to transfer the queue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61917362
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,190 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractProcessor{
    +
    +    public static final int LAST_WILL_QOS = 2;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                e.printStackTrace();
    --- End diff --
    
    You didn't mean that, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63537518
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    Yes, initiate drain when it is stopping. Basically use stop as onTrigger(..). At least that is what I would do. The more I think about it the more I think you should look at AbstractKafkaProcessor in pr/366 which handles very similar issues and may have applicability in a lot more cases then Kafka and in the future become one of the generic abstract processors that one can extend from.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63537926
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    --- End diff --
    
    @mosermw good point and is exactly what came out of the discussion I had with @markap14 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63578415
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    --- End diff --
    
    Same null check comment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63580606
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    --- End diff --
    
    I know its required, but since it is an expression that will be evaluated on FlowFile, can such evaluation return null? If so, then you get NPE.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63585234
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    --- End diff --
    
    It does, but between us two you are the one born with English, so I'll leave it up to you ;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63580163
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    --- End diff --
    
    Same response, PROP_QOS is required, used the same convention before and never had problems.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63580288
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RetainValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BrokerValidator)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    --- End diff --
    
    A general comment about use of NON_EMPTY_VALIDATOR. Given that this validator will pass as long as there is something entered including the white space(s), the question I have if all of the properties that are using it now are OK with that or they require NON_BLANK_VALIDATOR which will ensure that? I mean the above may all be correct, so I am just asking.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63583750
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    --- End diff --
    
    Yup I am in agreement with Joe, I just didn't do it myself. Will adjust.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63579787
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    --- End diff --
    
    We already have NON_BLANK_VALIDATOR in StandardValidators, so you may want to reuse.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61914386
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(!mqttClient.isConnected()){
    --- End diff --
    
    Hmm, I am questioning thread-safety here. Let's say two+ threads come to this line at the same time. It will be _true_ for all. Now, one goes and obtains connection while the others are blocking, then exits the sync block and let's another one to get in and so on with each reestablishing connection resulting in the connection leak.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61917246
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/SubscribeMQTT.java ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT"})
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({@WritesAttribute(attribute="broker", description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute="topic", description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute="qos", description="The quality of service for this message."),
    +    @WritesAttribute(attribute="isDuplicate", description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute="isRetained", description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published on the topic.")})
    +public class SubscribeMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .allowableValues("0", "1", "2")
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private String broker;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +    private ScheduledExecutorService autoReconnectExecutor;
    +    private final Object autoReconnectLock = new Object();
    +    private ScheduledFuture<?> autoReconnectScheduledFuture;
    +    private static int AUTO_RECONNECT_PERIOD = 5;
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +
    +    LinkedBlockingQueue<MQTTQueueMessage> mqttQueue = new LinkedBlockingQueue<>();
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +     private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC_FILTER);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_MAX_QUEUE_SIZE);
    +
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_MESSAGE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            String topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new SubscribeMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +                mqttClient.subscribe(topicFilter, context.getProperty(PROP_QOS).asInteger());
    +            }
    +            autoReconnectExecutor = new ScheduledThreadPoolExecutor(1);
    +        } catch(MqttException me) {
    +            getLogger().error("msg "+me.getMessage());
    +        }
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                }
    +            }
    +            synchronized (autoReconnectLock) {
    +                if(autoReconnectScheduledFuture != null && !autoReconnectScheduledFuture.isCancelled()) {
    +                    logger.debug("Stopping the scheduled future auto-reconnecter");
    +                    autoReconnectScheduledFuture.cancel(true);
    +                }
    +                if (autoReconnectExecutor != null) {
    +                    logger.debug("Shutting down the auto-reconnect executor service");
    +                    autoReconnectExecutor.shutdown();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    --- End diff --
    
    Also, as commented before, should we yield here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61917098
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/SubscribeMQTT.java ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT"})
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({@WritesAttribute(attribute="broker", description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute="topic", description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute="qos", description="The quality of service for this message."),
    +    @WritesAttribute(attribute="isDuplicate", description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute="isRetained", description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published on the topic.")})
    +public class SubscribeMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .allowableValues("0", "1", "2")
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private String broker;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +    private ScheduledExecutorService autoReconnectExecutor;
    +    private final Object autoReconnectLock = new Object();
    +    private ScheduledFuture<?> autoReconnectScheduledFuture;
    +    private static int AUTO_RECONNECT_PERIOD = 5;
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +
    +    LinkedBlockingQueue<MQTTQueueMessage> mqttQueue = new LinkedBlockingQueue<>();
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +     private List<PropertyDescriptor> descriptors;
    +
    +    private Set<Relationship> relationships;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC_FILTER);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_MAX_QUEUE_SIZE);
    +
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_MESSAGE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            String topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new SubscribeMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +                mqttClient.subscribe(topicFilter, context.getProperty(PROP_QOS).asInteger());
    +            }
    +            autoReconnectExecutor = new ScheduledThreadPoolExecutor(1);
    +        } catch(MqttException me) {
    +            getLogger().error("msg "+me.getMessage());
    +        }
    +    }
    --- End diff --
    
    As I stated above, the logic appears to be somewhat duplicated with the other processor. Perhaps some refactoring and moving all or part of it in the abstract class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r61948441
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    --- End diff --
    
    Yup will move


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63585228
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    --- End diff --
    
    I think re-using it would make it un-necessarily complicated. I will leave it as is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63581917
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RetainValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BrokerValidator)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    --- End diff --
    
    I don't want to impose limitations on passwords (who knows what people make their password) so I figure NON_EMPTY is valid. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63547530
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    This processor doesn't need to be scared of reconnecting when other tasks are running because the only time the onTrigger needs/touches the client is if/when it is reconnecting (and there is already lock around that). It isn't re-using the client multiple times in the OnTrigger like Kafka processors. For normal ConsumeMQTT onTriggers, it just transfers messages from the internal queue to the FlowFiles. 
    
    @OnScheduled is run when the processor is no longer scheduled (task may still be running) and @OnStopped is run when all the tasks are finished running. So we already get, for free, the ability to only drain as part of a graceful stop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63577875
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    --- End diff --
    
    Can this valuate to 'null' causing NPE on _asBoolan()_ call? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63578737
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    +
    +        try {
    +            /*
    +             * Underlying method waits for the message to publish (according to set QoS), so it executes synchronously:
    +             *     MqttClient.java:361 aClient.publish(topic, message, null, null).waitForCompletion(getTimeToWait());
    +             */
    +            mqttClient.publish(topic, mqttMessage);
    +            session.transfer(flowfile, REL_SUCCESS);
    +        } catch(MqttException me) {
    +            logger.error("Failed to publish message.", me);
    +            session.transfer(flowfile, REL_FAILURE);
    +        }
    +    }
    +
    +    private class PublishMQTTCallback  implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and re-connect failed");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.error("Message arrived to a PublishMQTT processor { topic:'" + topic +"; payload:"+ Arrays.toString(message.getPayload())+"}");
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            // Client.publish waits for message to be delivered so this token will always have a null message and is useless in this application.
    +            logger.debug("Received 'delivery complete' message from broker for:" + token.toString());
    +        }
    +    }
    +
    +
    +    private void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        }
    --- End diff --
    
    About 70-80% of code in this method is the same in both Publish and Consume variants. Perhaps some refactoring into a base method in abstract class would be in order.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63584682
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    --- End diff --
    
    The evaluation will always return a property value (even if it evaluates to a blank string)[1]. Then when it is evaluated as a boolean [2] it does a null check and passes it to Boolean.parseBoolean() which does another null check and returns false if the value is not equal to "true".
    
    [1] https://github.com/apache/nifi/blob/f378ee902127bd29b168c9bb15e991abe4eab0fa/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java#L137
    [2] https://github.com/apache/nifi/blob/f378ee902127bd29b168c9bb15e991abe4eab0fa/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java#L73


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61912184
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    --- End diff --
    
    Given NIFI-1384 and NIFI-1318 I would discourage doing the above  in the init method and instead use static initializer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63906843
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QOS_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BROKER_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RETAIN_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    --- End diff --
    
    Since its currently only supports tcp/ssl, i'd mentioned it in the comment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63907025
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QOS_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BROKER_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RETAIN_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    --- End diff --
    
    Definitely, will add


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r62033274
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,190 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractProcessor{
    +
    +    public static final int LAST_WILL_QOS = 2;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                e.printStackTrace();
    --- End diff --
    
    https://45.media.tumblr.com/9f90cb40ef2580146399b7588a33c966/tumblr_o0ose9Noon1rc0hxao1_400.gif
    
    What are you talking about ....?
    
    lol but yeah that's my bad


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63545947
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    +            if (mqttQueue.size() >= maxQueueSize){
    +                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
    +            } else {
    +                mqttQueue.add(new MQTTQueueMessage(topic, message));
    +            }
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            logger.warn("Received MQTT 'delivery complete' message to subscriber:"+ token);
    +        }
    +    }
    +
    +    // Public for testing
    +    public void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new ConsumeMQTTCallback());
    +                mqttClient.connect(connOpts);
    +                if(!subscribed){
    +                    mqttClient.subscribe(topicFilter, qos);
    +                    subscribed = true;
    +                }
    +            }
    +        }
    +    }
    +
    +    // Public for testing
    +    public boolean isConnected(){
    --- End diff --
    
    Do you have an example of using reflection purely for testing? When reviewing things I believe I've only ever seen it done this way.
    I understand and appreciate the need to be aware of making things public and breaking changes but I feel this is minimal in it's scope of impact. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63534793
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    --- End diff --
    
    Yup, you're correct. I will add a check for if it is scheduled or not.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on the pull request:

    https://github.com/apache/nifi/pull/392#issuecomment-219808772
  
    @JPercivall just a heads up; the branch now has merge conflicts


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63535091
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    --- End diff --
    
    General comment (applies to all logger callbacks) (except may be for _error_); While logger itself will handle the logging at the pre-configured level it is generally recommended to still wrap it in _IF_ bloc for cases where message String is composed (e.g., "foo" + this.getBar() + " bar") as you are essentially go through the process of building a message string that may never see the light of day. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63549263
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    +            if (mqttQueue.size() >= maxQueueSize){
    +                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
    +            } else {
    +                mqttQueue.add(new MQTTQueueMessage(topic, message));
    +            }
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            logger.warn("Received MQTT 'delivery complete' message to subscriber:"+ token);
    +        }
    +    }
    +
    +    // Public for testing
    +    public void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new ConsumeMQTTCallback());
    +                mqttClient.connect(connOpts);
    +                if(!subscribed){
    +                    mqttClient.subscribe(topicFilter, qos);
    +                    subscribed = true;
    +                }
    +            }
    +        }
    +    }
    +
    +    // Public for testing
    +    public boolean isConnected(){
    --- End diff --
    
    "minimal in it's scope of impact" - may be for today, but you can not possibly answer this question for tomorrow. In fact by making it public one can argue you've created the 'scope' (however minimal it may be)  when it didn't even have to exist in the first place.
    As for an example just get an instance of 'mqttClient' instance using the _Class.getDeclaredField_ dance and perform the same evaluation.
    ```
    ConsumeMQTT mqtt = ...
    Field f = mqtt.getClass().getDeclaredField("mqttClient");
    f.setAccessible(true);
    MqttClient mqttClient = f.get(mqtt);
    // do your check on mqttClient
    ```
    Put it in some private method within your test cases (some abstract class or some util class) and you're done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63918531
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QOS_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BROKER_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RETAIN_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BROKER_VALIDATOR)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder().name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Last Will Topic")
    +            .description("The topic to send the client's Last Will to. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_MESSAGE = new PropertyDescriptor.Builder()
    +            .name("Last Will Message")
    +            .description("The message to send as the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Last Will Retain")
    +            .description("Whether to retain the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .allowableValues("true","false")
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_QOS = new PropertyDescriptor.Builder()
    +            .name("Last Will QoS Level")
    +            .description("QoS level to be used when publishing the Last Will Message")
    +            .required(false)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2
    +            )
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CLEAN_SESSION = new PropertyDescriptor.Builder()
    +            .name("Session state")
    +            .description("Whether to start afresh or resume previous flows. See the allowable value descriptions for more details.")
    +            .required(true)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_TRUE,
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_FALSE
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_CLEAN_SESSION_TRUE.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MQTT_VERSION = new PropertyDescriptor.Builder()
    +            .name("MQTT Specification Version")
    +            .description("The MQTT specification version when connecting with the broker. See the allowable value descriptions for more details.")
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_MQTT_VERSION_AUTO,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_311,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_310
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_MQTT_VERSION_AUTO.getValue())
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CONN_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Connection Timeout (seconds)")
    +            .description("Maximum time interval the client will wait for the network connection to the MQTT server " +
    +                    "to be established. The default timeout is 30 seconds. " +
    +                    "A value of 0 disables timeout processing meaning the client will wait until the network connection is made successfully or fails.")
    +            .required(false)
    +            .defaultValue("30")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive Interval (seconds)")
    +            .description("Defines the maximum time interval between messages sent or received. It enables the " +
    +                    "client to detect if the server is no longer available, without having to wait for the TCP/IP timeout. " +
    +                    "The client will ensure that at least one message travels across the network within each keep alive period. In the absence of a data-related message during the time period, " +
    +                    "the client sends a very small \"ping\" message, which the server will acknowledge. A value of 0 disables keepalive processing in the client.")
    +            .required(false)
    +            .defaultValue("60")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static List<PropertyDescriptor> getAbstractPropertyDescriptors(){
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(PROP_BROKER_URI);
    +        descriptors.add(PROP_CLIENTID);
    +        descriptors.add(PROP_USERNAME);
    +        descriptors.add(PROP_PASSWORD);
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(PROP_LAST_WILL_TOPIC);
    +        descriptors.add(PROP_LAST_WILL_MESSAGE);
    +        descriptors.add(PROP_LAST_WILL_RETAIN);
    +        descriptors.add(PROP_LAST_WILL_QOS);
    +        descriptors.add(PROP_CLEAN_SESSION);
    +        descriptors.add(PROP_MQTT_VERSION);
    +        descriptors.add(PROP_CONN_TIMEOUT);
    +        descriptors.add(PROP_KEEP_ALIVE_INTERVAL);
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
    +        final List<ValidationResult> results = new ArrayList<>(1);
    +        final boolean usernameSet = validationContext.getProperty(PROP_USERNAME).isSet();
    +        final boolean passwordSet = validationContext.getProperty(PROP_PASSWORD).isSet();
    +
    +        if ((usernameSet && !passwordSet) || (!usernameSet && passwordSet)) {
    +            results.add(new ValidationResult.Builder().subject("Username and Password").valid(false).explanation("if username or password is set, both must be set").build());
    +        }
    +
    +        final boolean lastWillTopicSet = validationContext.getProperty(PROP_LAST_WILL_TOPIC).isSet();
    +        final boolean lastWillMessageSet = validationContext.getProperty(PROP_LAST_WILL_MESSAGE).isSet();
    +
    +        final boolean lastWillRetainSet = validationContext.getProperty(PROP_LAST_WILL_RETAIN).isSet();
    +        final boolean lastWillQosSet = validationContext.getProperty(PROP_LAST_WILL_QOS).isSet();
    +
    +        // If any of the Last Will Properties are set
    +        if(lastWillTopicSet || lastWillMessageSet || lastWillRetainSet || lastWillQosSet){
    +            // And any are not set
    +            if(!(lastWillTopicSet && lastWillMessageSet && lastWillRetainSet && lastWillQosSet)){
    +                // Then mark as invalid
    +                results.add(new ValidationResult.Builder().subject("Last Will Properties").valid(false).explanation("if any of the Last Will Properties (message, topic, retain and QoS) are " +
    +                        "set, all must be set.").build());
    +            }
    +        }
    +
    +        return results;
    +    }
    +
    +    public static Properties transformSSLContextService(SSLContextService sslContextService){
    +        Properties properties = new Properties();
    +        properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
    --- End diff --
    
    Great idea, will add that information


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63583956
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    --- End diff --
    
    Something like that (if possible). Just to avoid duplication of code. That's the only argument, unless using it will prove more complicated. Your call.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63224728
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    --- End diff --
    
    I realize I'm really late to this discussion, but flowfile can be null on an INPUT_REQUIRED processor is when it's configured with >1 Concurrent Tasks.  If 1 flowfile arrives in an input queue, then the framework can sometimes assign more than 1 thread to run this processor's onTrigger().  The first thread will get the flowfile and the second thread will get a null flowfile and should just return.  The framework won't schedule this processor again until another flowfile arrives on an input queue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63580066
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    --- End diff --
    
    PROP_RETAIN is required. So as far as I know, it can never evaluate to Null and I've used (and seen others use) the same convention and not had any problems.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63558222
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    Yup OnUnscheduled (I'm assuming you meant "unscheuled" note "schedule") is invoked before OnStopped but I'm not using the client in OnStopped (the resource that gets closed in OnUnscheduled). The flow of logic is:
    OnUnScheduled: close the connection and stop receiving messages
    remaining onTrigger tasks finish
    OnStopped: if any messages are left in the internal message queue, drain them


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63533161
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    Also, not sure about this. I know we talked about it and I think in the event where 'finish processing the messages' was configured what should happen is that you should allow them to drain but not accept any more requests. 
    I am looking at it from the user's perspective. Upon seeng this exception what would the user do? How would they fix it? Basically what I am saying is that there is no way to stop this processor clean if there are messages in the queue unless you configure it to ignore those essentially resulting in message loss.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63917779
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QOS_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RETAIN_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are transferred to this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are transferred to this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    --- End diff --
    
    I just did some testing and I don't see any reason to synchronize anywhere but onTrigger() (the only place where where multi-threading is possible), so I would recommend revisiting and if agreed removing it.
    Other then that I think we are getting close to a merge. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63902327
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private volatile long maxQueueSize;
    +
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +    private AtomicBoolean scheduled = new AtomicBoolean(false);
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.warn("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request. Processor will be invalid.",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +        scheduled.set(true);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        scheduled.set(false);
    +
    +        try {
    +            synchronized (mqttClientConnectLock) {
    --- End diff --
    
    May be I am missing something, but what's the point of synchronization here? Is there a contention of some type? I mean knowing when/how @OnUnscheduled is invoked i don't seem to see the benefit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63608020
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        if (mqttQueue.isEmpty() && !isConnected()){
    +            logger.info("Queue is empty and client is not connected. Attempting to reconnect.");
    +
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost (or was never connected) and ontrigger connect failed. Yielding processor", e);
    +                context.yield();
    +            }
    +        }
    +
    +        if (mqttQueue.isEmpty()) {
    +            return;
    +        }
    +
    +        transferQueue(session);
    +    }
    +
    +    private void transferQueue(ProcessSession session){
    +        while (!mqttQueue.isEmpty()) {
    +            FlowFile messageFlowfile = session.create();
    +            final MQTTQueueMessage mqttMessage = mqttQueue.peek();
    +
    +            Map<String, String> attrs = new HashMap<>();
    +            attrs.put(BROKER_ATTRIBUTE_KEY, broker);
    +            attrs.put(TOPIC_ATTRIBUTE_KEY, mqttMessage.getTopic());
    +            attrs.put(QOS_ATTRIBUTE_KEY, String.valueOf(mqttMessage.getQos()));
    +            attrs.put(IS_DUPLICATE_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isDuplicate()));
    +            attrs.put(IS_RETAINED_ATTRIBUTE_KEY, String.valueOf(mqttMessage.isRetained()));
    +
    +            messageFlowfile = session.putAllAttributes(messageFlowfile, attrs);
    +
    +            messageFlowfile = session.write(messageFlowfile, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    out.write(mqttMessage.getPayload());
    +                }
    +            });
    +
    +            String transitUri = new StringBuilder(broker).append(mqttMessage.getTopic()).toString();
    +            session.getProvenanceReporter().receive(messageFlowfile, transitUri);
    +            session.transfer(messageFlowfile, REL_MESSAGE);
    +            mqttQueue.remove(mqttMessage);
    +            session.commit();
    +        }
    +    }
    +
    +    private class ConsumeMQTTCallback implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and callback re-connect failed.");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.info("MQTT message arrived on topic:" + topic);
    +            if (mqttQueue.size() >= maxQueueSize){
    +                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
    +            } else {
    +                mqttQueue.add(new MQTTQueueMessage(topic, message));
    +            }
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            logger.warn("Received MQTT 'delivery complete' message to subscriber:"+ token);
    +        }
    +    }
    +
    +    // Public for testing
    +    public void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new ConsumeMQTTCallback());
    +                mqttClient.connect(connOpts);
    +                if(!subscribed){
    +                    mqttClient.subscribe(topicFilter, qos);
    +                    subscribed = true;
    +                }
    +            }
    +        }
    +    }
    +
    +    // Public for testing
    +    public boolean isConnected(){
    --- End diff --
    
    I adjusted to use reflection in a couple private helper methods


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63903305
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private volatile long maxQueueSize;
    +
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +    private AtomicBoolean scheduled = new AtomicBoolean(false);
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.warn("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request. Processor will be invalid.",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +        scheduled.set(true);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        scheduled.set(false);
    +
    +        try {
    +            synchronized (mqttClientConnectLock) {
    --- End diff --
    
    You don't want to try and call the "disconnect" when the mqttClient is getting remade in a "setAndReconnect" which can get called in the OnTrigger (which could be running at the same time). Need to synchronize any time there are changes to mqttClient.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61916950
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/SubscribeMQTT.java ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.ScheduledThreadPoolExecutor;
    +import java.util.concurrent.TimeUnit;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT"})
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({@WritesAttribute(attribute="broker", description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute="topic", description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute="qos", description="The quality of service for this message."),
    +    @WritesAttribute(attribute="isDuplicate", description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute="isRetained", description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published on the topic.")})
    +public class SubscribeMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .allowableValues("0", "1", "2")
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private String broker;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +    private ScheduledExecutorService autoReconnectExecutor;
    +    private final Object autoReconnectLock = new Object();
    +    private ScheduledFuture<?> autoReconnectScheduledFuture;
    +    private static int AUTO_RECONNECT_PERIOD = 5;
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +
    +    LinkedBlockingQueue<MQTTQueueMessage> mqttQueue = new LinkedBlockingQueue<>();
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +     private List<PropertyDescriptor> descriptors;
    --- End diff --
    
    Seem to be some formatting error. It's indented an extra space.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61915046
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    --- End diff --
    
    I see this pattern a lot, but technically it;s a bit wrong as it will spin without any delay (yield) if FlowFile is null. And while I am not a fan of "GO TO" style (issuing 'return' in the middle of code flow), should you at least issue ```context.yield()``` before return?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r61949743
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    --- End diff --
    
    PublishMQTT won't spin "without any delay (yield)" because it requires input and under normal circumstances will only get triggered by the controller when there are flowfiles to process. This check is more or less a fail-safe. Yielding is for when there is something wrong with the processor for which waiting may help resolve (failing to reconnect). I don't see this as an instance that warrants yielding.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63919932
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QOS_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RETAIN_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are transferred to this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are transferred to this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    --- End diff --
    
    But PublishMQTT doesn't have onUnscheduled, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63919082
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QOS_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BROKER_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RETAIN_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BROKER_VALIDATOR)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder().name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Last Will Topic")
    +            .description("The topic to send the client's Last Will to. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_MESSAGE = new PropertyDescriptor.Builder()
    +            .name("Last Will Message")
    +            .description("The message to send as the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Last Will Retain")
    +            .description("Whether to retain the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .allowableValues("true","false")
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_QOS = new PropertyDescriptor.Builder()
    +            .name("Last Will QoS Level")
    +            .description("QoS level to be used when publishing the Last Will Message")
    +            .required(false)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2
    +            )
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CLEAN_SESSION = new PropertyDescriptor.Builder()
    +            .name("Session state")
    +            .description("Whether to start afresh or resume previous flows. See the allowable value descriptions for more details.")
    +            .required(true)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_TRUE,
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_FALSE
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_CLEAN_SESSION_TRUE.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MQTT_VERSION = new PropertyDescriptor.Builder()
    +            .name("MQTT Specification Version")
    +            .description("The MQTT specification version when connecting with the broker. See the allowable value descriptions for more details.")
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_MQTT_VERSION_AUTO,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_311,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_310
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_MQTT_VERSION_AUTO.getValue())
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CONN_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Connection Timeout (seconds)")
    +            .description("Maximum time interval the client will wait for the network connection to the MQTT server " +
    +                    "to be established. The default timeout is 30 seconds. " +
    +                    "A value of 0 disables timeout processing meaning the client will wait until the network connection is made successfully or fails.")
    +            .required(false)
    +            .defaultValue("30")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive Interval (seconds)")
    +            .description("Defines the maximum time interval between messages sent or received. It enables the " +
    +                    "client to detect if the server is no longer available, without having to wait for the TCP/IP timeout. " +
    +                    "The client will ensure that at least one message travels across the network within each keep alive period. In the absence of a data-related message during the time period, " +
    +                    "the client sends a very small \"ping\" message, which the server will acknowledge. A value of 0 disables keepalive processing in the client.")
    +            .required(false)
    +            .defaultValue("60")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static List<PropertyDescriptor> getAbstractPropertyDescriptors(){
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(PROP_BROKER_URI);
    +        descriptors.add(PROP_CLIENTID);
    +        descriptors.add(PROP_USERNAME);
    +        descriptors.add(PROP_PASSWORD);
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(PROP_LAST_WILL_TOPIC);
    +        descriptors.add(PROP_LAST_WILL_MESSAGE);
    +        descriptors.add(PROP_LAST_WILL_RETAIN);
    +        descriptors.add(PROP_LAST_WILL_QOS);
    +        descriptors.add(PROP_CLEAN_SESSION);
    +        descriptors.add(PROP_MQTT_VERSION);
    +        descriptors.add(PROP_CONN_TIMEOUT);
    +        descriptors.add(PROP_KEEP_ALIVE_INTERVAL);
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
    +        final List<ValidationResult> results = new ArrayList<>(1);
    +        final boolean usernameSet = validationContext.getProperty(PROP_USERNAME).isSet();
    +        final boolean passwordSet = validationContext.getProperty(PROP_PASSWORD).isSet();
    +
    +        if ((usernameSet && !passwordSet) || (!usernameSet && passwordSet)) {
    +            results.add(new ValidationResult.Builder().subject("Username and Password").valid(false).explanation("if username or password is set, both must be set").build());
    +        }
    +
    +        final boolean lastWillTopicSet = validationContext.getProperty(PROP_LAST_WILL_TOPIC).isSet();
    +        final boolean lastWillMessageSet = validationContext.getProperty(PROP_LAST_WILL_MESSAGE).isSet();
    +
    +        final boolean lastWillRetainSet = validationContext.getProperty(PROP_LAST_WILL_RETAIN).isSet();
    +        final boolean lastWillQosSet = validationContext.getProperty(PROP_LAST_WILL_QOS).isSet();
    +
    +        // If any of the Last Will Properties are set
    +        if(lastWillTopicSet || lastWillMessageSet || lastWillRetainSet || lastWillQosSet){
    +            // And any are not set
    +            if(!(lastWillTopicSet && lastWillMessageSet && lastWillRetainSet && lastWillQosSet)){
    +                // Then mark as invalid
    +                results.add(new ValidationResult.Builder().subject("Last Will Properties").valid(false).explanation("if any of the Last Will Properties (message, topic, retain and QoS) are " +
    +                        "set, all must be set.").build());
    +            }
    +        }
    +
    +        return results;
    +    }
    +
    +    public static Properties transformSSLContextService(SSLContextService sslContextService){
    +        Properties properties = new Properties();
    +        properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
    --- End diff --
    
    For consistency sake you may want to look how it is done in new Kafka https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java#L100


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63941204
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QOS_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BROKER_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RETAIN_VALIDATOR = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BROKER_VALIDATOR)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder().name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Last Will Topic")
    +            .description("The topic to send the client's Last Will to. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_MESSAGE = new PropertyDescriptor.Builder()
    +            .name("Last Will Message")
    +            .description("The message to send as the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Last Will Retain")
    +            .description("Whether to retain the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .allowableValues("true","false")
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_QOS = new PropertyDescriptor.Builder()
    +            .name("Last Will QoS Level")
    +            .description("QoS level to be used when publishing the Last Will Message")
    +            .required(false)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2
    +            )
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CLEAN_SESSION = new PropertyDescriptor.Builder()
    +            .name("Session state")
    +            .description("Whether to start afresh or resume previous flows. See the allowable value descriptions for more details.")
    +            .required(true)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_TRUE,
    +                    ALLOWABLE_VALUE_CLEAN_SESSION_FALSE
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_CLEAN_SESSION_TRUE.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MQTT_VERSION = new PropertyDescriptor.Builder()
    +            .name("MQTT Specification Version")
    +            .description("The MQTT specification version when connecting with the broker. See the allowable value descriptions for more details.")
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_MQTT_VERSION_AUTO,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_311,
    +                    ALLOWABLE_VALUE_MQTT_VERSION_310
    +            )
    +            .defaultValue(ALLOWABLE_VALUE_MQTT_VERSION_AUTO.getValue())
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_CONN_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Connection Timeout (seconds)")
    +            .description("Maximum time interval the client will wait for the network connection to the MQTT server " +
    +                    "to be established. The default timeout is 30 seconds. " +
    +                    "A value of 0 disables timeout processing meaning the client will wait until the network connection is made successfully or fails.")
    +            .required(false)
    +            .defaultValue("30")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_KEEP_ALIVE_INTERVAL = new PropertyDescriptor.Builder()
    +            .name("Keep Alive Interval (seconds)")
    +            .description("Defines the maximum time interval between messages sent or received. It enables the " +
    +                    "client to detect if the server is no longer available, without having to wait for the TCP/IP timeout. " +
    +                    "The client will ensure that at least one message travels across the network within each keep alive period. In the absence of a data-related message during the time period, " +
    +                    "the client sends a very small \"ping\" message, which the server will acknowledge. A value of 0 disables keepalive processing in the client.")
    +            .required(false)
    +            .defaultValue("60")
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static List<PropertyDescriptor> getAbstractPropertyDescriptors(){
    +        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
    +        descriptors.add(PROP_BROKER_URI);
    +        descriptors.add(PROP_CLIENTID);
    +        descriptors.add(PROP_USERNAME);
    +        descriptors.add(PROP_PASSWORD);
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(PROP_LAST_WILL_TOPIC);
    +        descriptors.add(PROP_LAST_WILL_MESSAGE);
    +        descriptors.add(PROP_LAST_WILL_RETAIN);
    +        descriptors.add(PROP_LAST_WILL_QOS);
    +        descriptors.add(PROP_CLEAN_SESSION);
    +        descriptors.add(PROP_MQTT_VERSION);
    +        descriptors.add(PROP_CONN_TIMEOUT);
    +        descriptors.add(PROP_KEEP_ALIVE_INTERVAL);
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
    +        final List<ValidationResult> results = new ArrayList<>(1);
    +        final boolean usernameSet = validationContext.getProperty(PROP_USERNAME).isSet();
    +        final boolean passwordSet = validationContext.getProperty(PROP_PASSWORD).isSet();
    +
    +        if ((usernameSet && !passwordSet) || (!usernameSet && passwordSet)) {
    +            results.add(new ValidationResult.Builder().subject("Username and Password").valid(false).explanation("if username or password is set, both must be set").build());
    +        }
    +
    +        final boolean lastWillTopicSet = validationContext.getProperty(PROP_LAST_WILL_TOPIC).isSet();
    +        final boolean lastWillMessageSet = validationContext.getProperty(PROP_LAST_WILL_MESSAGE).isSet();
    +
    +        final boolean lastWillRetainSet = validationContext.getProperty(PROP_LAST_WILL_RETAIN).isSet();
    +        final boolean lastWillQosSet = validationContext.getProperty(PROP_LAST_WILL_QOS).isSet();
    +
    +        // If any of the Last Will Properties are set
    +        if(lastWillTopicSet || lastWillMessageSet || lastWillRetainSet || lastWillQosSet){
    +            // And any are not set
    +            if(!(lastWillTopicSet && lastWillMessageSet && lastWillRetainSet && lastWillQosSet)){
    +                // Then mark as invalid
    +                results.add(new ValidationResult.Builder().subject("Last Will Properties").valid(false).explanation("if any of the Last Will Properties (message, topic, retain and QoS) are " +
    +                        "set, all must be set.").build());
    +            }
    +        }
    +
    +        return results;
    +    }
    +
    +    public static Properties transformSSLContextService(SSLContextService sslContextService){
    +        Properties properties = new Properties();
    +        properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
    --- End diff --
    
    Actually I don't think this information will be useful to the user. This is because the these property names ("com.ibm.ssl.protocol") are specific to the underlying library that is used (Paho MQTT) and are only implementation details. For MQTT in general, they are the same as what are in the Context Service. So it would be more confusing to the user to explain "SSL Protocol -> com.ibm.ssl.protocol" for each property instead of just having the property names in the service during set up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63920230
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QOS_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RETAIN_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are transferred to this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are transferred to this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    --- End diff --
    
    Actually I should probably a introduce read/write lock for PublishMQTT on the MqttClient instead of just a basic lock


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63579264
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    +
    +        try {
    +            /*
    +             * Underlying method waits for the message to publish (according to set QoS), so it executes synchronously:
    +             *     MqttClient.java:361 aClient.publish(topic, message, null, null).waitForCompletion(getTimeToWait());
    +             */
    +            mqttClient.publish(topic, mqttMessage);
    +            session.transfer(flowfile, REL_SUCCESS);
    +        } catch(MqttException me) {
    +            logger.error("Failed to publish message.", me);
    +            session.transfer(flowfile, REL_FAILURE);
    +        }
    +    }
    +
    +    private class PublishMQTTCallback  implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and re-connect failed");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.error("Message arrived to a PublishMQTT processor { topic:'" + topic +"; payload:"+ Arrays.toString(message.getPayload())+"}");
    --- End diff --
    
    This is intentional because this is Publisher and really shouldn't ever receive a message. If it does then something weird is happening.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63530121
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    --- End diff --
    
    Hate to be picky, but given that this is set in @OnSchduled but used elsewhere (other threads) it needs to be _volatile_.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63582187
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RetainValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BrokerValidator)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    --- End diff --
    
    Fair enough, just wanted to make sure you look at all other PDs where it is used and if any of them must actually have a non-blank value.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63585339
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,239 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({ConsumeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'. " +
    +                    "Expression language is allowed in order to support publishing messages with different QoS but the end value of the property must be either '0', '1' or '2'. ")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static {
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_RETAIN);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<>();
    +        innerRelationshipsSet.add(REL_SUCCESS);
    +        innerRelationshipsSet.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        buildClient(context);
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        if(mqttClient == null || !mqttClient.isConnected()){
    +            logger.info("Was disconnected from client or was never connected, attempting to connect.");
    +            synchronized (mqttClientConnectLock){
    +                try {
    +                    reconnect();
    +                } catch (MqttException e) {
    +                    context.yield();
    +                    session.transfer(flowfile, REL_FAILURE);
    +                    logger.error("MQTT client is disconnected and re-connecting failed. Transferring FlowFile to fail and yielding", e);
    +                    return;
    +                }
    +            }
    +        }
    +
    +        // get the MQTT topic
    +        String topic = context.getProperty(PROP_TOPIC).evaluateAttributeExpressions(flowfile).getValue();
    +
    +        if (topic == null || topic.isEmpty()) {
    +            logger.warn("Evaluation of the topic property returned null or evaluated to be empty, routing to failure");
    +            session.transfer(flowfile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // do the read
    +        final byte[] messageContent = new byte[(int) flowfile.getSize()];
    +        session.read(flowfile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, messageContent, true);
    +            }
    +        });
    +
    +        int qos = context.getProperty(PROP_QOS).evaluateAttributeExpressions(flowfile).asInteger();
    +        final MqttMessage mqttMessage = new MqttMessage(messageContent);
    +        mqttMessage.setQos(qos);
    +        mqttMessage.setPayload(messageContent);
    +        mqttMessage.setRetained(context.getProperty(PROP_RETAIN).evaluateAttributeExpressions(flowfile).asBoolean());
    +
    +        try {
    +            /*
    +             * Underlying method waits for the message to publish (according to set QoS), so it executes synchronously:
    +             *     MqttClient.java:361 aClient.publish(topic, message, null, null).waitForCompletion(getTimeToWait());
    +             */
    +            mqttClient.publish(topic, mqttMessage);
    +            session.transfer(flowfile, REL_SUCCESS);
    +        } catch(MqttException me) {
    +            logger.error("Failed to publish message.", me);
    +            session.transfer(flowfile, REL_FAILURE);
    +        }
    +    }
    +
    +    private class PublishMQTTCallback  implements MqttCallback {
    +
    +        @Override
    +        public void connectionLost(Throwable cause) {
    +            logger.warn("Connection to " + broker + " lost", cause);
    +            try {
    +                reconnect();
    +            } catch (MqttException e) {
    +                logger.error("Connection to " + broker + " lost and re-connect failed");
    +            }
    +        }
    +
    +        @Override
    +        public void messageArrived(String topic, MqttMessage message) throws Exception {
    +            logger.error("Message arrived to a PublishMQTT processor { topic:'" + topic +"; payload:"+ Arrays.toString(message.getPayload())+"}");
    +        }
    +
    +        @Override
    +        public void deliveryComplete(IMqttDeliveryToken token) {
    +            // Client.publish waits for message to be delivered so this token will always have a null message and is useless in this application.
    +            logger.debug("Received 'delivery complete' message from broker for:" + token.toString());
    +        }
    +    }
    +
    +
    +    private void reconnect() throws MqttException {
    +        synchronized (mqttClientConnectLock) {
    +            if (!mqttClient.isConnected()) {
    +                mqttClient = getMqttClient(broker, clientID, persistence);
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        }
    --- End diff --
    
    Agreed, I will refactor these three lines to their own method in the abstract class:
    ```
            mqttClient = getMqttClient(broker, clientID, persistence);
            mqttClient.setCallback(mqttCallback);
            mqttClient.connect(connOpts);
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63551055
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private long maxQueueSize;
    +
    +    private volatile boolean subscribed = false;
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +
    +    private volatile LinkedBlockingQueue<MQTTQueueMessage> mqttQueue;
    +
    +    // For Testing
    +    public LinkedBlockingQueue<MQTTQueueMessage> getMqttQueue(){
    +        return mqttQueue;
    +    }
    +
    +    public static final Relationship REL_MESSAGE = new Relationship.Builder()
    +            .name("Message")
    +            .description("The MQTT message output")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> descriptors;
    +    private static final Set<Relationship> relationships;
    +
    +    static{
    +        final List<PropertyDescriptor> innerDescriptorsList = getAbstractPropertyDescriptors();
    +        innerDescriptorsList.add(PROP_TOPIC_FILTER);
    +        innerDescriptorsList.add(PROP_QOS);
    +        innerDescriptorsList.add(PROP_MAX_QUEUE_SIZE);
    +        descriptors = Collections.unmodifiableList(innerDescriptorsList);
    +
    +        final Set<Relationship> innerRelationshipsSet = new HashSet<Relationship>();
    +        innerRelationshipsSet.add(REL_MESSAGE);
    +        relationships = Collections.unmodifiableSet(innerRelationshipsSet);
    +    }
    +
    +    @Override
    +    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
    +        // resize the receive buffer, but preserve data
    +        if (descriptor == PROP_MAX_QUEUE_SIZE) {
    +            // it's a mandatory integer, never null
    +            int newSize = Integer.valueOf(newValue);
    +            if (mqttQueue != null) {
    +                int msgPending = mqttQueue.size();
    +                if (msgPending > newSize) {
    +                    logger.debug("New receive buffer size ({}) is smaller than the number of messages pending ({}), ignoring resize request",
    +                            new Object[]{newSize, msgPending});
    +                    return;
    +                }
    +                LinkedBlockingQueue<MQTTQueueMessage> newBuffer = new LinkedBlockingQueue<>(newSize);
    +                mqttQueue.drainTo(newBuffer);
    +                mqttQueue = newBuffer;
    +            }
    +
    +        }
    +    }
    +
    +    @Override
    +    public Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = super.customValidate(context);
    +        int newSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger();
    +        if (mqttQueue == null) {
    +            mqttQueue = new LinkedBlockingQueue<>(context.getProperty(PROP_MAX_QUEUE_SIZE).asInteger());
    +        }
    +        int msgPending = mqttQueue.size();
    +        if (msgPending > newSize) {
    +            results.add(new ValidationResult.Builder()
    +                    .valid(false)
    +                    .subject("ConsumeMQTT Configuration")
    +                    .explanation(String.format("%s (%d) is smaller than the number of messages pending (%d).",
    +                            PROP_MAX_QUEUE_SIZE.getDisplayName(), newSize, msgPending))
    +                    .build());
    +        }
    +
    +        return results;
    +    }
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) throws IOException, ClassNotFoundException {
    +        qos = context.getProperty(PROP_QOS).asInteger();
    +        maxQueueSize = context.getProperty(PROP_MAX_QUEUE_SIZE).asLong();
    +        topicFilter = context.getProperty(PROP_TOPIC_FILTER).getValue();
    +
    +        buildClient(context);
    +    }
    +
    +    @OnUnscheduled
    +    public void onUnscheduled(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientConnectLock) {
    +                if(mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect(DISCONNECT_TIMEOUT);
    +                    logger.info("Disconnected the MQTT client.");
    +                }
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +
    +    @OnStopped
    +    public void onStopped(final ProcessContext context) throws IOException {
    +        if(processSessionFactory != null) {
    +            logger.info("Finishing processing leftover messages");
    +            ProcessSession session = processSessionFactory.createSession();
    +            transferQueue(session);
    +        } else {
    +            if (mqttQueue!= null && mqttQueue.size() > 0){
    +                throw new ProcessException("Attempting to stop processor but stored ProcessSessionFactory is not set, there are messages in the MQTT Queue and it is configured to " +
    +                        "finish processing the messages.");
    +            }
    --- End diff --
    
    May be, and I'll let you deside what's right/wrong. Just wanted to point out the concenrns that I have as an observer which I still do, since while current implementation may be tailored to the current design, any potential changes to the code have to take into account how everything else works as opposed to have separation between life-cycle and operation of the processor. 
     
    As for the second point, look at StandardProcessNode:1324. Unless I am missing something @OnSchedule is invoked before @OnStopped which means you are closing resources while they may still be in use.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r61956949
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    --- End diff --
    
    Then we should discuss it and address it properly. Probably addressing NIFI-53 first


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r61950939
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    +        } catch(MqttException me) {
    +            logger.error("Failed when disconnecting the MQTT client.", me);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    --- End diff --
    
    Fair enough even though I don't like it ;) Basically we need to either guarantee it from the core framework (proc won't be called unless there are input files) or yield ;)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61937600
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    --- End diff --
    
    Good call, I will move them to a static initializer


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808

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

    https://github.com/apache/nifi/pull/392#discussion_r61913545
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.io.InputStream;
    +import java.io.IOException;
    +
    +@SupportsBatching
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"publish", "MQTT", "IOT"})
    +@CapabilityDescription("Publishes a message to an MQTT topic")
    +@SeeAlso({SubscribeMQTT.class})
    +public class PublishMQTT extends AbstractMQTTProcessor {
    +
    +    private String broker;
    +    private MemoryPersistence persistence = new MemoryPersistence();
    +    private MqttClient mqttClient;
    +    private final Object mqttClientLock = new Object();
    +
    +    public static final PropertyDescriptor PROP_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Topic")
    +            .description("The topic to publish the message to.")
    +            .expressionLanguageSupported(true)
    +            .required(true)
    +            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(QoSValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Retain Message")
    +            .description("Whether or not the retain flag should be set on the MQTT message.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(RetainValidator)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("QoS Timeout")
    +            .description("How many milliseconds to wait for the broker to acknowledge delivery of a message before routing the message back to the source queue for retry. Only applicable " +
    +                    "when QoS is 1 or 2")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    private List<PropertyDescriptor> descriptors;
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("FlowFiles that are sent successfully to the destination are sent out this relationship.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("FlowFiles that failed to send to the destination are sent out this relationship.")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private ProcessorLog logger;
    +    private MqttConnectOptions connOpts;
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        logger = getLogger();
    +        final List<PropertyDescriptor> descriptors = getAbstractPropertyDescriptors();
    +        descriptors.add(PROP_TOPIC);
    +        descriptors.add(PROP_QOS);
    +        descriptors.add(PROP_RETAIN);
    +        descriptors.add(PROP_TIMEOUT);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<Relationship>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return this.relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        try {
    +            broker = context.getProperty(PROP_BROKER_URI).getValue();
    +            String clientID = context.getProperty(PROP_CLIENTID).getValue();
    +
    +            connOpts = new MqttConnectOptions();
    +            connOpts.setCleanSession(true);
    +
    +            PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
    +            if (sslProp.isSet()) {
    +                Properties sslProps = transformSSLContextService((SSLContextService) sslProp.asControllerService());
    +                connOpts.setSSLProperties(sslProps);
    +            }
    +
    +            PropertyValue lastWillTopicProp = context.getProperty(PROP_LAST_WILL_TOPIC);
    +            if (lastWillTopicProp.isSet()){
    +                PropertyValue lastWillMessage = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_TOPIC);
    +                connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getValue().getBytes(), LAST_WILL_QOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
    +            }
    +
    +            PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
    +            if(usernameProp.isSet()) {
    +                connOpts.setUserName(usernameProp.getValue());
    +                connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
    +            }
    +
    +            synchronized (mqttClientLock) {
    +                mqttClient = new MqttClient(broker, clientID, persistence);
    +
    +                mqttClient.setCallback(new PublishMQTTCallback());
    +                getLogger().info("Connecting to broker: " + broker);
    +                mqttClient.connect(connOpts);
    +            }
    +        } catch(MqttException me) {
    +            logger.error("Failed to initialize the connection to the  "+me.getMessage());
    +        }
    +    }
    +
    +    @OnStopped
    +    public void onStop(final ProcessContext context) {
    +        try {
    +            synchronized (mqttClientLock) {
    +                if (mqttClient != null && mqttClient.isConnected()) {
    +                    mqttClient.disconnect();
    +                }
    +            }
    +            logger.info("Disconnected the MQTT client.");
    --- End diff --
    
    Should the above log be moved inside of IF?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63539083
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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.mqtt.common;
    +
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.expression.AttributeExpression;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.eclipse.paho.client.mqttv3.IMqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttClient;
    +import org.eclipse.paho.client.mqttv3.MqttConnectOptions;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
    +
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Properties;
    +
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_FALSE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_CLEAN_SESSION_TRUE;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_310;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_311;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_MQTT_VERSION_AUTO;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProcessor {
    +
    +    protected ProcessorLog logger;
    +    protected IMqttClient mqttClient;
    +    protected final Object mqttClientConnectLock = new Object();
    +    protected volatile String broker;
    +    protected volatile String clientID;
    +    protected MqttConnectOptions connOpts;
    +    protected MemoryPersistence persistence = new MemoryPersistence();
    +    protected long maxTimeout;
    +
    +    public ProcessSessionFactory processSessionFactory;
    +
    +    public static final Validator QoSValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            Integer inputInt = Integer.parseInt(input);
    +            if (inputInt < 0 || inputInt > 2) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("QoS must be an integer between 0 and 2").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator BrokerValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            try{
    +                URI brokerURI = new URI(input);
    +                if (!"".equals(brokerURI.getPath())) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is:" + brokerURI.getPath()).build();
    +                }
    +                if (!("tcp".equals(brokerURI.getScheme()) || "ssl".equals(brokerURI.getScheme()))) {
    +                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp' and 'ssl' schemes are supported.").build();
    +                }
    +            } catch (URISyntaxException e) {
    +                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
    +            }
    +            return new ValidationResult.Builder().subject(subject).valid(true).build();
    +        }
    +    };
    +
    +    public static final Validator RetainValidator = new Validator() {
    +
    +        @Override
    +        public ValidationResult validate(String subject, String input, ValidationContext context) {
    +            if("true".equalsIgnoreCase(input) || "false".equalsIgnoreCase(input)){
    +                return new ValidationResult.Builder().subject(subject).valid(true).build();
    +            } else{
    +                return StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.BOOLEAN, false)
    +                        .validate(subject, input, context);
    +            }
    +
    +        }
    +    };
    +
    +    public static final PropertyDescriptor PROP_BROKER_URI = new PropertyDescriptor.Builder()
    +            .name("Broker URI")
    +            .description("The URI to use to connect to the MQTT broker (e.g. tcp://localhost:1883)")
    +            .required(true)
    +            .addValidator(BrokerValidator)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_CLIENTID = new PropertyDescriptor.Builder()
    +            .name("Client ID")
    +            .description("MQTT client ID to use")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to use when connecting to the broker")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder().name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_TOPIC = new PropertyDescriptor.Builder()
    +            .name("Last Will Topic")
    +            .description("The topic to send the client's Last Will to. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_MESSAGE = new PropertyDescriptor.Builder()
    +            .name("Last Will Message")
    +            .description("The message to send as the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_RETAIN = new PropertyDescriptor.Builder()
    +            .name("Last Will Retain")
    +            .description("Whether to retain the client's Last Will. If the Last Will topic and message are not set then a Last Will will not be sent.")
    +            .required(false)
    +            .allowableValues("true","false")
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_LAST_WILL_QOS = new PropertyDescriptor.Builder()
    +            .name("'Last Will' QoS Level")
    +            .description("QoS level to be used when publishing the Will Message")
    +            .required(false)
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2
    +            )
    --- End diff --
    
    Should here be a default value or is this intentional?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1808 Creating general MQTT processors

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

    https://github.com/apache/nifi/pull/392#discussion_r63902702
  
    --- Diff: nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * 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.mqtt;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.TriggerSerially;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessorInitializationContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor;
    +import org.apache.nifi.processors.mqtt.common.MQTTQueueMessage;
    +import org.eclipse.paho.client.mqttv3.MqttCallback;
    +import org.eclipse.paho.client.mqttv3.MqttException;
    +import org.eclipse.paho.client.mqttv3.MqttMessage;
    +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.io.OutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.BROKER_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_DUPLICATE_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.IS_RETAINED_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.QOS_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.ConsumeMQTT.TOPIC_ATTRIBUTE_KEY;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_0;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_1;
    +import static org.apache.nifi.processors.mqtt.common.MqttConstants.ALLOWABLE_VALUE_QOS_2;
    +
    +
    +@Tags({"subscribe", "MQTT", "IOT", "consume", "listen"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@TriggerSerially // we want to have a consistent mapping between clientID and MQTT connection
    +@CapabilityDescription("Subscribes to a topic and receives messages from an MQTT broker")
    +@SeeAlso({PublishMQTT.class})
    +@WritesAttributes({
    +    @WritesAttribute(attribute=BROKER_ATTRIBUTE_KEY, description="MQTT broker that was the message source"),
    +    @WritesAttribute(attribute=TOPIC_ATTRIBUTE_KEY, description="MQTT topic on which message was received"),
    +    @WritesAttribute(attribute=QOS_ATTRIBUTE_KEY, description="The quality of service for this message."),
    +    @WritesAttribute(attribute=IS_DUPLICATE_ATTRIBUTE_KEY, description="Whether or not this message might be a duplicate of one which has already been received."),
    +    @WritesAttribute(attribute=IS_RETAINED_ATTRIBUTE_KEY, description="Whether or not this message was from a current publisher, or was \"retained\" by the server as the last message published " +
    +            "on the topic.")})
    +public class ConsumeMQTT extends AbstractMQTTProcessor {
    +
    +    public final static String BROKER_ATTRIBUTE_KEY =  "mqtt.broker";
    +    public final static String TOPIC_ATTRIBUTE_KEY =  "mqtt.topic";
    +    public final static String QOS_ATTRIBUTE_KEY =  "mqtt.qos";
    +    public final static String IS_DUPLICATE_ATTRIBUTE_KEY =  "mqtt.isDuplicate";
    +    public final static String IS_RETAINED_ATTRIBUTE_KEY =  "mqtt.isRetained";
    +
    +    public static final PropertyDescriptor PROP_TOPIC_FILTER = new PropertyDescriptor.Builder()
    +            .name("Topic Filter")
    +            .description("The MQTT topic filter to designate the topics to subscribe to.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_QOS = new PropertyDescriptor.Builder()
    +            .name("Quality of Service(QoS)")
    +            .description("The Quality of Service(QoS) to receive the message with. Accepts values '0', '1' or '2'; '0' for 'at most once', '1' for 'at least once', '2' for 'exactly once'.")
    +            .required(true)
    +            .defaultValue(ALLOWABLE_VALUE_QOS_0.getValue())
    +            .allowableValues(
    +                    ALLOWABLE_VALUE_QOS_0,
    +                    ALLOWABLE_VALUE_QOS_1,
    +                    ALLOWABLE_VALUE_QOS_2)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_MAX_QUEUE_SIZE = new PropertyDescriptor.Builder()
    +            .name("Max Queue Size")
    +            .description("The MQTT messages are always being sent to subscribers on a topic. If the 'Run Schedule' is significantly behind the rate at which the messages are arriving to this " +
    +                    "processor then a back up can occur. This property specifies the maximum number of messages this processor will hold in memory at one time.")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +
    +    private static int DISCONNECT_TIMEOUT = 5000;
    +    private volatile long maxQueueSize;
    +
    +    private volatile int qos;
    +    private volatile String topicFilter;
    +    private AtomicBoolean scheduled = new AtomicBoolean(false);
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---