You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/07/19 21:56:13 UTC

[GitHub] [nifi] nandorsoma opened a new pull request, #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

nandorsoma opened a new pull request, #6225:
URL: https://github.com/apache/nifi/pull/6225

   <!-- Licensed to the Apache Software Foundation (ASF) under one or more -->
   <!-- contributor license agreements.  See the NOTICE file distributed with -->
   <!-- this work for additional information regarding copyright ownership. -->
   <!-- The ASF licenses this file to You under the Apache License, Version 2.0 -->
   <!-- (the "License"); you may not use this file except in compliance with -->
   <!-- the License.  You may obtain a copy of the License at -->
   <!--     http://www.apache.org/licenses/LICENSE-2.0 -->
   <!-- Unless required by applicable law or agreed to in writing, software -->
   <!-- distributed under the License is distributed on an "AS IS" BASIS, -->
   <!-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -->
   <!-- See the License for the specific language governing permissions and -->
   <!-- limitations under the License. -->
   
   # Summary
   
   [NIFI-10251](https://issues.apache.org/jira/browse/NIFI-10251)
   This pr adds v5 protocol support for existing MQTT processors. For v5 connections the processor from now on uses HiveMQ Client library while in case of v3.1.x connections it uses the existing Paho library. HiveMQ Client could have been used for v3.1.x connections but it seemed to be safer to use the existing library for compatibility reasons.
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [x] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [x] Build completed using `mvn clean install -P contrib-check`
     - [x] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [x] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [x] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952894025


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();

Review Comment:
   It could affect performance and the old Paho v3 library was also working in an async way. But as we agreed I'm changing it to blocking mode in case of Hive v5, because it is more important to route failed messages to the failed relationship.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941319081


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -199,7 +199,7 @@ public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProces
             .build();
 
     public static final PropertyDescriptor PROP_SESSION_EXPIRY_INTERVAL = new PropertyDescriptor.Builder()
-            .name("Session Expiry Interval")
+            .name("Session Expiry Interval (seconds)")

Review Comment:
   Changing the property name breaks backward compatibility, so this change needs to be reverted. It is possible to change the `displayName`, but in this case, I think it is better to rely on the description instead of adding hint to the property name.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r956209009


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final long DEFAULT_SESSION_EXPIRY_INTERVAL_IN_SECONDS = 3600;

Review Comment:
   I don't know. In v3 it was infinite, that's true, but it was also considered as a bug which led to stucking sessions. That's why v5 implemented that feature. For me, under normal circumstances 1 hour seems to be a reasonable value, something above that feels too long. But I cannot prove it in any way. I think when NiFi is down for 1 hour then it is either an expected situation when the administrator needs to think about that (eg. stopping the source system that sends the message) or in an unexpected situation the session timeout would barely really matter.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957810380


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java:
##########
@@ -201,35 +191,35 @@ private void initializeClient(ProcessContext context) {
         // non-null but not connected, so we need to handle each case and only create a new client when it is null
         try {
             if (mqttClient == null) {
-                logger.debug("Creating client");
-                mqttClient = createMqttClient(broker, clientID, persistence);
+                mqttClient = createMqttClient();
                 mqttClient.setCallback(this);
             }
 
             if (!mqttClient.isConnected()) {
-                logger.debug("Connecting client");
-                mqttClient.connect(connOpts);
+                mqttClient.connect();
             }
-        } catch (MqttException e) {
-            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
+        } catch (Exception e) {
+            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", clientProperties.getBroker(), e);
             context.yield();
         }
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);
     }
 
     @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())+"}");
+    public void messageArrived(ReceivedMqttMessage message) {
+        // Unlikely situation. Api uses the same callback for publisher and consumer as well.
+        // That's why we have this log message here to indicate something really messy thing happened.
+        logger.error("Message arrived to a PublishMQTT processor { topic:'" + message.getTopic() + "; payload:" + Arrays.toString(message.getPayload()) + "}");
     }
 
     @Override
-    public void deliveryComplete(IMqttDeliveryToken token) {
+    public void deliveryComplete(String 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.trace("Received 'delivery complete' message from broker for:" + token.toString());
+        logger.trace("Received 'delivery complete' message from broker for:" + token);

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957817027


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java:
##########
@@ -649,38 +625,44 @@ private void closeWriter(final RecordSetWriter writer) {
     }
 
     private String getTransitUri(String... appends) {
-        StringBuilder stringBuilder = new StringBuilder(brokerUri);
-        for(String append : appends) {
+        String broker = clientProperties.getBrokerUri().toString();
+        StringBuilder stringBuilder = new StringBuilder(broker.endsWith("/") ? broker : broker + "/");
+        for (String append : appends) {
             stringBuilder.append(append);
         }
         return stringBuilder.toString();
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);
     }
 
     @Override
-    public void messageArrived(String topic, MqttMessage message) throws Exception {
+    public void messageArrived(ReceivedMqttMessage message) {
         if (logger.isDebugEnabled()) {
             byte[] payload = message.getPayload();
-            String text = new String(payload, "UTF-8");
+            String text = new String(payload, StandardCharsets.UTF_8);
             if (StringUtils.isAsciiPrintable(text)) {
-                logger.debug("Message arrived from topic {}. Payload: {}", new Object[] {topic, text});
+                logger.debug("Message arrived from topic {}. Payload: {}", message.getTopic(), text);
             } else {
-                logger.debug("Message arrived from topic {}. Binary value of size {}", new Object[] {topic, payload.length});
+                logger.debug("Message arrived from topic {}. Binary value of size {}", message.getTopic(), payload.length);
             }
         }
 
-        if(!mqttQueue.offer(new MQTTQueueMessage(topic, message), 1, TimeUnit.SECONDS)) {
-            throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
+        try {
+            if (!mqttQueue.offer(message, 1, TimeUnit.SECONDS)) {
+                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
+            }
+        } catch (InterruptedException e) {
+            throw new MqttException("Failed to process message arrived from topic " + message.getTopic());
         }
     }
 
     @Override
-    public void deliveryComplete(IMqttDeliveryToken token) {
-        logger.warn("Received MQTT 'delivery complete' message to subscriber: " + token);
+    public void deliveryComplete(String token) {
+        // Unlikely situation. Api uses the same callback for publisher and consumer as well.
+        // That's why we have this log message here to indicate something really messy thing happened.
+        logger.error("Received MQTT 'delivery complete' message to subscriber: " + token);

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r945591904


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {

Review Comment:
   I just declared them to make it easier to follow where Paho client expects an exception. My thought was that this way it  is easier to make sure that the behavior is the same as the original solution with the old client. (That was true for every implementation decision.) Nevertheless, I've removed them.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r950100231


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);
+
+        private final int numericValue;

Review Comment:
   `versionCode` may be more descriptive.
   Its type could be `String` because it is always be used as `String`.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();

Review Comment:
   I think we need to publish in blocking mode (no `mqtt5Client.toAsync()`). Otherwise we will not be notified when a publish is unsuccessful, will we?
   The returned `Future` could be used for it but blocking mode seems to me simpler.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttClientFactory.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5ClientBuilder;
+import org.apache.nifi.processors.mqtt.adapters.HiveMqV5ClientAdapter;
+import org.apache.nifi.processors.mqtt.adapters.PahoMqttClientAdapter;
+import org.eclipse.paho.client.mqttv3.MqttClient;
+import org.eclipse.paho.client.mqttv3.MqttException;
+import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
+
+import static org.apache.nifi.processors.mqtt.common.MqttConstants.SupportedSchemes.SSL;
+import static org.apache.nifi.processors.mqtt.common.MqttConstants.SupportedSchemes.WS;
+import static org.apache.nifi.processors.mqtt.common.MqttConstants.SupportedSchemes.WSS;
+
+public class MqttClientFactory {
+    public NifiMqttClient create(MqttClientProperties clientProperties, MqttConnectionProperties connectionProperties) {
+        switch (clientProperties.getMqttVersion()) {
+            case 0:
+            case 3:
+            case 4:
+                return createPahoMqttV3ClientAdapter(clientProperties);
+            case 5:
+                return createHiveMqV5ClientAdapter(clientProperties, connectionProperties);

Review Comment:
   Using some constants could to be useful here.
   `MqttVersion` enum may be used in `MqttClientProperties` instead of `int`, though it does not contain the AUTO option currently.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);

Review Comment:
   Values 3 and 4 come from the Paho library (`MqttConnectOptions.MQTT_VERSION_3_1` and `MqttConnectOptions.MQTT_VERSION_3_1_1`). I think it would make sense to reference those constants here.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttClientProperties.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 java.net.URI;
+
+public class MqttClientProperties {
+    private URI brokerURI;
+    private String clientID;

Review Comment:
   Minor: Following the conventions you use in the PR, these should be `brokerUri` and `clientId`. Also `MqttConnectionProperties.lastWillQos`.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);
+
+        private final int numericValue;
+
+        MqttVersion(int numericValue) {
+            this.numericValue = numericValue;
+        }
+
+        public int getNumericValue() {
+            return numericValue;
+        }
+    }
+
+    public enum SupportedSchemes {

Review Comment:
   Please use singular for enum names (like `MqttVersion` above).
   I would also omit the "Supported" prefix. `MqttScheme` or `MqttProtocolScheme` sound better to me.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/NifiMqttMessage.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+public class NifiMqttMessage {

Review Comment:
   Do we really need it as a separate class besides `MQTTQueueMessage`?
   `MQTTQueueMessage` wraps a topic name and the fields from `NifiMqttMessage`. In the places where `NifiMqttMessage` is constructed, the topic is always present and they travel together to the point where `MQTTQueueMessage` is created. Could we just create a single object with all the topic and other fields at the beginning?



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);
+                    }
+                })
+                .send();

Review Comment:
   The result of the `subscribe()` call could be checked via the returned `Future` in order to provide an error message on the processor's bulletin. If I understand it correctly, the error message is only logged by the asynchronous thread currently.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();

Review Comment:
   Not clear the intent for me here. Does it make sense to pass a password in without a username? 



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);

Review Comment:
   The client is already in blocking mode (ensured by the constructor) so no further `toBlocking()` call would be needed (like in `isConnected()` method).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941155396


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -199,16 +198,13 @@ public ValidationResult validate(String subject, String input, ValidationContext
             .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)
+    public static final PropertyDescriptor PROP_SESSION_EXPIRY_INTERVAL = new PropertyDescriptor.Builder()
+            .name("Session Expiry Interval")
+            .description("After this interval the broker will expire the client and clear the session state.")
+            .addValidator(StandardValidators.NON_NEGATIVE_LONG_VALIDATOR)
+            .dependsOn(PROP_MQTT_VERSION, ALLOWABLE_VALUE_MQTT_VERSION_500)
+            .dependsOn(PROP_CLEAN_SESSION, ALLOWABLE_VALUE_CLEAN_SESSION_FALSE)
+            .defaultValue(Long.toString(SESSION_EXPIRY_INTERVAL_IN_SECONDS))
             .build();

Review Comment:
   Thanks for the suggestion @tpalfy ! I like this solution however there are similar properties in the processor where just numbers without units were used. I think it is better to stick to the original version to avoid mixing the input types. Nevertheless I've added a hint to the name of the property to match the other properties.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r945709768


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);

Review Comment:
   That sounds good, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r953140061


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -289,88 +288,35 @@ public Collection<ValidationResult> customValidate(final ValidationContext valid
         return results;
     }
 
-    public static Properties transformSSLContextService(SSLContextService sslContextService){
-        Properties properties = new Properties();
-        if (sslContextService.getSslAlgorithm() != null) {
-            properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
-        }
-        if (sslContextService.getKeyStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.keyStore", sslContextService.getKeyStoreFile());
-        }
-        if (sslContextService.getKeyStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.keyStorePassword", sslContextService.getKeyStorePassword());
-        }
-        if (sslContextService.getKeyStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.keyStoreType", sslContextService.getKeyStoreType());
-        }
-        if (sslContextService.getTrustStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.trustStore", sslContextService.getTrustStoreFile());
-        }
-        if (sslContextService.getTrustStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.trustStorePassword", sslContextService.getTrustStorePassword());
-        }
-        if (sslContextService.getTrustStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.trustStoreType", sslContextService.getTrustStoreType());
-        }
-        return  properties;
-    }
-
-    protected void onScheduled(final ProcessContext context){
-        broker = context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue();
-        brokerUri = broker.endsWith("/") ? broker : broker + "/";
-        clientID = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
-
-        if (clientID == null) {
-            clientID = UUID.randomUUID().toString();
-        }
-
-        connOpts = new MqttConnectOptions();
-        connOpts.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
-        connOpts.setKeepAliveInterval(context.getProperty(PROP_KEEP_ALIVE_INTERVAL).asInteger());
-        connOpts.setMqttVersion(context.getProperty(PROP_MQTT_VERSION).asInteger());
-        connOpts.setConnectionTimeout(context.getProperty(PROP_CONN_TIMEOUT).asInteger());
-
-        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()){
-            String lastWillMessage = context.getProperty(PROP_LAST_WILL_MESSAGE).getValue();
-            PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_RETAIN);
-            Integer lastWillQOS = context.getProperty(PROP_LAST_WILL_QOS).asInteger();
-            connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getBytes(), lastWillQOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
-        }
-
-
-        PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
-        if(usernameProp.isSet()) {
-            connOpts.setUserName(usernameProp.evaluateAttributeExpressions().getValue());
-            connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
-        }
+    protected void onScheduled(final ProcessContext context) {
+        clientProperties = getMqttClientProperties(context);
+        connectionProperties = getMqttConnectionProperties(context);
     }
 
     protected void onStopped() {
-        try {
-            logger.info("Disconnecting client");
-            mqttClient.disconnect(DISCONNECT_TIMEOUT);
-        } catch(MqttException me) {
-            logger.error("Error disconnecting MQTT client due to {}", new Object[]{me.getMessage()}, me);
-        }
+        // Since client is created in the onTrigger method it can happen that it never will be created because of an initialization error.
+        // We are preventing additional nullPtrException here, but the clean solution would be to create the client in the onScheduled method.
+        if (mqttClient != null) {
+            try {
+                logger.info("Disconnecting client");
+                mqttClient.disconnect(DISCONNECT_TIMEOUT);
+            } catch (MqttException me) {
+                logger.error("Error disconnecting MQTT client due to {}", new Object[]{me.getMessage()}, me);
+            }
+
+            try {
+                logger.info("Closing client");
+                mqttClient.close();
+            } catch (MqttException me) {
+                logger.error("Error closing MQTT client due to {}", new Object[]{me.getMessage()}, me);
+            }

Review Comment:
   All exceptions should be caught here to ensure `mqttClient = null` to be executed in all cases.
   I can still run into the issue that `mqttClient` is not cleared properly and therefore it will not be reinitalized. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952879128


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);
+                    }
+                })
+                .send();
+    }
+
+    @Override
+    public void setCallback(NifiMqttCallback callback) {
+        this.callback = callback;
+    }
+
+    public static KeyManagerFactory getKeyManagerFactory(String keyStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore keyStore = loadIntoKeyStore(keyStoreType, path, keyStorePassword);
+
+            final KeyManagerFactory kmf = KeyManagerFactory
+                    .getInstance(KeyManagerFactory.getDefaultAlgorithm());
+            kmf.init(keyStore, new char[0]); // https://stackoverflow.com/questions/1814048/sun-java-keymanagerfactory-and-null-passwords
+
+            return kmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static TrustManagerFactory getTrustManagerFactory(String trustStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore trustStore = loadIntoKeyStore(trustStoreType, path, keyStorePassword);
+
+            final TrustManagerFactory tmf = TrustManagerFactory
+                    .getInstance(TrustManagerFactory.getDefaultAlgorithm());
+            tmf.init(trustStore);
+
+            return tmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static KeyStore loadIntoKeyStore(String type, String path, char[] keyStorePassword) throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+        final KeyStore keyStore = KeyStore.getInstance(type);
+        final InputStream in = new FileInputStream(path);
+        keyStore.load(in, keyStorePassword);
+        return keyStore;
+    }
+
+    private byte[] toBytes(char[] chars) {
+        final ByteBuffer byteBuffer = StandardCharsets.UTF_8.encode(CharBuffer.wrap(chars));
+        final byte[] bytes = Arrays.copyOfRange(byteBuffer.array(), byteBuffer.position(), byteBuffer.limit());
+        clearSensitive(byteBuffer.array());
+        return bytes;
+    }
+
+    private void clearSensitive(char[] chars) {

Review Comment:
   Changed, thanks for noticing it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952888865


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();

Review Comment:
   Mqtt v5 allows to use just password without username: https://www.hivemq.com/blog/mqtt5-essentials-part2-foundational-changes-in-the-protocol/ (see Using passwords without usernames)
   But as we agreed, now I'm adding username to the check and when needed we will create a separate field on the processor to allow authentication just with password.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r954027026


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.mqtt.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import com.hivemq.client.mqtt.mqtt5.message.subscribe.suback.Mqtt5SubAck;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.mqtt.common.MqttCallback;
+import org.apache.nifi.processors.mqtt.common.MqttClient;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.MqttException;
+import org.apache.nifi.processors.mqtt.common.ReceivedMqttMessage;
+import org.apache.nifi.processors.mqtt.common.StandardMqttMessage;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+
+public class HiveMqV5ClientAdapter implements MqttClient {
+
+    private final Mqtt5BlockingClient mqtt5BlockingClient;
+    private final ComponentLog logger;
+
+    private MqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient, ComponentLog logger) {
+        this.mqtt5BlockingClient = mqtt5BlockingClient;
+        this.logger = logger;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5BlockingClient.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) {
+        logger.debug("Connecting to broker");
+
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        final String username = connectionProperties.getUsername();
+        final String password = connectionProperties.getPassword();
+        if (username != null && password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(password.getBytes(StandardCharsets.UTF_8))
+                    .applySimpleAuth();
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5BlockingClient.connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) {
+        logger.debug("Disconnecting client");
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5BlockingClient.disconnect();
+    }
+
+    @Override
+    public void close() {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, StandardMqttMessage message) {
+        logger.debug("Publishing message to {} with QoS: {}", topic, message.getQos());
+
+        mqtt5BlockingClient.publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        logger.debug("Subscribing to {} with QoS: {}", topicFilter, qos);
+
+        CompletableFuture<Mqtt5SubAck> ack = mqtt5BlockingClient.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final ReceivedMqttMessage receivedMessage = new ReceivedMqttMessage(
+                            mqtt5Publish.getPayloadAsBytes(),
+                            mqtt5Publish.getQos().getCode(),
+                            mqtt5Publish.isRetain(),
+                            mqtt5Publish.getTopic().toString());
+                    callback.messageArrived(receivedMessage);
+                })
+                .send();
+
+        // Setting "listener" callback is only possible with async client, though sending subscribe message
+        // should happen in a blocking way to make sure the processor is blocked until ack is not arrived.
+        ack.whenComplete((mqtt5SubAck, throwable) -> {
+            logger.debug("Received mqtt5 subscribe ack: {}", mqtt5SubAck.toString());
+
+            if (throwable != null) {
+                throw new MqttException("An error has occurred during sending subscribe message to broker", throwable);
+            }
+        });

Review Comment:
   It confused me that there is also whenCompleteAsync and because of that I thought whenComplete blocks. Thanks for noticing it! Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on PR #6225:
URL: https://github.com/apache/nifi/pull/6225#issuecomment-1227128897

   After a discussion I've removed commit (NIFI-10251 use ReceivedMqttMessageHandler instead of a generic callback which was mostly used by Paho and in the 2/3 of the cases it didn't require callback [52204e4]). Probably I will open a separate pr for that change.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957339014


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final long DEFAULT_SESSION_EXPIRY_INTERVAL_IN_SECONDS = 3600;

Review Comment:
   Just a little addition to my "feels too long" comment. If I set the session timeout to that long, I have to think about how these messages will be stored on the broker (thousands of messages / sec for 24 hours) and then how these messages will be processed along with the new messages. Will NiFi be able to keep up with that temporary increased load? Ofc on the other side there is the problem of loosing data... Changing to 24 hours as we agreed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957816155


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java:
##########
@@ -649,38 +625,44 @@ private void closeWriter(final RecordSetWriter writer) {
     }
 
     private String getTransitUri(String... appends) {
-        StringBuilder stringBuilder = new StringBuilder(brokerUri);
-        for(String append : appends) {
+        String broker = clientProperties.getBrokerUri().toString();
+        StringBuilder stringBuilder = new StringBuilder(broker.endsWith("/") ? broker : broker + "/");
+        for (String append : appends) {
             stringBuilder.append(append);
         }
         return stringBuilder.toString();
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957824765


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();
             }
-            return new ValidationResult.Builder().subject(subject).valid(true).build();
+            if (!isValidEnumIgnoreCase(MqttProtocolScheme.class, brokerURI.getScheme())) {
+                return new ValidationResult.Builder().subject(subject).valid(false)
+                        .explanation("invalid scheme! supported schemes are: " + MqttProtocolScheme.getValuesAsString(", ")).build();

Review Comment:
   Double colon feels weird in the same sentence. What about comma?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952888865


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();

Review Comment:
   Mqtt v5 allows to use just password without username: https://www.hivemq.com/blog/mqtt5-essentials-part2-foundational-changes-in-the-protocol/ (see Using passwords without usernames)
   But as we agreed, now I'll add username to the check and when needed we will create a separate field on the processor to allow authentication just with password.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r954027026


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.mqtt.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import com.hivemq.client.mqtt.mqtt5.message.subscribe.suback.Mqtt5SubAck;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.mqtt.common.MqttCallback;
+import org.apache.nifi.processors.mqtt.common.MqttClient;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.MqttException;
+import org.apache.nifi.processors.mqtt.common.ReceivedMqttMessage;
+import org.apache.nifi.processors.mqtt.common.StandardMqttMessage;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+
+public class HiveMqV5ClientAdapter implements MqttClient {
+
+    private final Mqtt5BlockingClient mqtt5BlockingClient;
+    private final ComponentLog logger;
+
+    private MqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient, ComponentLog logger) {
+        this.mqtt5BlockingClient = mqtt5BlockingClient;
+        this.logger = logger;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5BlockingClient.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) {
+        logger.debug("Connecting to broker");
+
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        final String username = connectionProperties.getUsername();
+        final String password = connectionProperties.getPassword();
+        if (username != null && password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(password.getBytes(StandardCharsets.UTF_8))
+                    .applySimpleAuth();
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5BlockingClient.connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) {
+        logger.debug("Disconnecting client");
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5BlockingClient.disconnect();
+    }
+
+    @Override
+    public void close() {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, StandardMqttMessage message) {
+        logger.debug("Publishing message to {} with QoS: {}", topic, message.getQos());
+
+        mqtt5BlockingClient.publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        logger.debug("Subscribing to {} with QoS: {}", topicFilter, qos);
+
+        CompletableFuture<Mqtt5SubAck> ack = mqtt5BlockingClient.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final ReceivedMqttMessage receivedMessage = new ReceivedMqttMessage(
+                            mqtt5Publish.getPayloadAsBytes(),
+                            mqtt5Publish.getQos().getCode(),
+                            mqtt5Publish.isRetain(),
+                            mqtt5Publish.getTopic().toString());
+                    callback.messageArrived(receivedMessage);
+                })
+                .send();
+
+        // Setting "listener" callback is only possible with async client, though sending subscribe message
+        // should happen in a blocking way to make sure the processor is blocked until ack is not arrived.
+        ack.whenComplete((mqtt5SubAck, throwable) -> {
+            logger.debug("Received mqtt5 subscribe ack: {}", mqtt5SubAck.toString());
+
+            if (throwable != null) {
+                throw new MqttException("An error has occurred during sending subscribe message to broker", throwable);
+            }
+        });

Review Comment:
   It confused me that there is also `whenCompleteAsync` and because of that I thought `whenComplete` blocks. Thanks for noticing it! Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957824765


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();
             }
-            return new ValidationResult.Builder().subject(subject).valid(true).build();
+            if (!isValidEnumIgnoreCase(MqttProtocolScheme.class, brokerURI.getScheme())) {
+                return new ValidationResult.Builder().subject(subject).valid(false)
+                        .explanation("invalid scheme! supported schemes are: " + MqttProtocolScheme.getValuesAsString(", ")).build();

Review Comment:
   Double colon feels weird in the same sentence. What about comma? Btw in this class most of the explanation starts with lowercase. What is the rule, which approach should I follow?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] tpalfy commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r940489917


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-nar/src/main/resources/META-INF/NOTICE:
##########
@@ -41,6 +41,11 @@ The following binary components are provided under the Apache Software License v
       in some artifacts (usually source distributions); but is always available
       from the source code management (SCM) system project uses.
 
+  (ASLv2) HiveMQ MQTT Client

Review Comment:
   This is good but I think we need to add it to nifi-assembly/NOTICE as well.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -199,16 +198,13 @@ public ValidationResult validate(String subject, String input, ValidationContext
             .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)
+    public static final PropertyDescriptor PROP_SESSION_EXPIRY_INTERVAL = new PropertyDescriptor.Builder()
+            .name("Session Expiry Interval")
+            .description("After this interval the broker will expire the client and clear the session state.")
+            .addValidator(StandardValidators.NON_NEGATIVE_LONG_VALIDATOR)
+            .dependsOn(PROP_MQTT_VERSION, ALLOWABLE_VALUE_MQTT_VERSION_500)
+            .dependsOn(PROP_CLEAN_SESSION, ALLOWABLE_VALUE_CLEAN_SESSION_FALSE)
+            .defaultValue(Long.toString(SESSION_EXPIRY_INTERVAL_IN_SECONDS))
             .build();

Review Comment:
   I'd rather not add the quite redundant and mostly copy-pasted `StandardValidators.NON_NEGATIVE_LONG_VALIDATOR`. We could use the more user-friendly and readily available `.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)` functionality.
   
   ```suggestion
       public static final PropertyDescriptor PROP_SESSION_EXPIRY_INTERVAL = new PropertyDescriptor.Builder()
               .name("Session Expiry Interval")
               .description("After this interval the broker will expire the client and clear the session state.")
               .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
               .dependsOn(PROP_MQTT_VERSION, ALLOWABLE_VALUE_MQTT_VERSION_500)
               .dependsOn(PROP_CLEAN_SESSION, ALLOWABLE_VALUE_CLEAN_SESSION_FALSE)
               .defaultValue(SESSION_EXPIRY_INTERVAL_IN_SECONDS + " sec")
               .build();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on PR #6225:
URL: https://github.com/apache/nifi/pull/6225#issuecomment-1197699462

   @nandorsoma Thanks for adding v5 protocol support to MQTT processors!
   
   I started to review / test this PR and found that the v5 client cannot stop properly:
   ```
   2022-07-28 07:53:40,084 ERROR [Timer-Driven Process Thread-4] org.apache.nifi.util.ReflectionUtils Failed while invoking annotated method 'public void org.apache.nifi.processors.mqtt.ConsumeMQTT.onUnscheduled(org.apache.nifi.processor.ProcessContext)' with arguments '[org.apache.nifi.processor.StandardProcessContext@5f360d4e]'.
   com.hivemq.client.mqtt.exceptions.MqttClientStateException: MQTT client is not connected.
   	at com.hivemq.client.internal.mqtt.MqttBlockingClient.disconnect(MqttBlockingClient.java:195)
   	at com.hivemq.client.internal.mqtt.MqttBlockingClient.disconnect(MqttBlockingClient.java:186)
   	at org.apache.nifi.processors.mqtt.paho.HiveMqV5ClientAdapter.close(HiveMqV5ClientAdapter.java:99)
   	at org.apache.nifi.processors.mqtt.common.AbstractMQTTProcessor.onStopped(AbstractMQTTProcessor.java:292)
   	at org.apache.nifi.processors.mqtt.ConsumeMQTT.onUnscheduled(ConsumeMQTT.java:348)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.apache.nifi.util.ReflectionUtils.invokeMethodsWithAnnotations(ReflectionUtils.java:145)
   	at org.apache.nifi.util.ReflectionUtils.invokeMethodsWithAnnotations(ReflectionUtils.java:133)
   	at org.apache.nifi.util.ReflectionUtils.quietlyInvokeMethodsWithAnnotations(ReflectionUtils.java:316)
   	at org.apache.nifi.util.ReflectionUtils.quietlyInvokeMethodsWithAnnotation(ReflectionUtils.java:93)
   	at org.apache.nifi.controller.StandardProcessorNode$2.run(StandardProcessorNode.java:1877)
   	at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   ```
   Could you please fix it first? Due to this error, the processor cannot create a new client and NiFi restart needed.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r954994656


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final long DEFAULT_SESSION_EXPIRY_INTERVAL_IN_SECONDS = 3600;

Review Comment:
   I feel the 3600 seconds (1 hour) default value a bit strict. It means that if NiFi is down for 1 hour, the consume processor will loose messages. I would set it to 24 hours or so (in v3 it was infinite).



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java:
##########
@@ -402,42 +398,27 @@ private void initializeClient(ProcessContext context) {
         // non-null but not connected, so we need to handle each case and only create a new client when it is null
         try {
             if (mqttClient == null) {
-                logger.debug("Creating client");
-                mqttClient = createMqttClient(broker, clientID, persistence);
+                mqttClient = createMqttClient();
                 mqttClient.setCallback(this);
             }
 
             if (!mqttClient.isConnected()) {
-                logger.debug("Connecting client");
-                mqttClient.connect(connOpts);
+                mqttClient.connect();
                 mqttClient.subscribe(topicPrefix + topicFilter, qos);
             }
-        } catch (MqttException e) {
-            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
+        } catch (Exception e) {
+            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{clientProperties.getBroker()}, e);
+            mqttClient = null; // prevent stucked processor when subscribe fails

Review Comment:
   I recommend to execute the same logic as in case of `OnStopped` in order to prevent resource leaking due to clients which fail to subscribe but already connected successfully (and remains on the heap until it gets garbage collected). 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on PR #6225:
URL: https://github.com/apache/nifi/pull/6225#issuecomment-1191244151

   > Thanks for the contribution @nandorsoma!
   > 
   > On initial review, the `Static Analysis` check failed due to multiple files missing the standard Apache License header. Please review the output of that check and add the header to the files indicated.
   
   Interesting, because I've run the build with `contrib-check` enabled which I thought checks for that. Nevertheless I will add them of course!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r956186934


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java:
##########
@@ -402,42 +398,27 @@ private void initializeClient(ProcessContext context) {
         // non-null but not connected, so we need to handle each case and only create a new client when it is null
         try {
             if (mqttClient == null) {
-                logger.debug("Creating client");
-                mqttClient = createMqttClient(broker, clientID, persistence);
+                mqttClient = createMqttClient();
                 mqttClient.setCallback(this);
             }
 
             if (!mqttClient.isConnected()) {
-                logger.debug("Connecting client");
-                mqttClient.connect(connOpts);
+                mqttClient.connect();
                 mqttClient.subscribe(topicPrefix + topicFilter, qos);
             }
-        } catch (MqttException e) {
-            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
+        } catch (Exception e) {
+            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{clientProperties.getBroker()}, e);
+            mqttClient = null; // prevent stucked processor when subscribe fails

Review Comment:
   Thanks for the idea, changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952880574


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/NifiMqttClient.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+public interface NifiMqttClient {

Review Comment:
   I went with MqttClient without prefix. Hope it is good that way!



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/NifiMqttMessage.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+public class NifiMqttMessage {

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952881239


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);
+
+        private final int numericValue;
+
+        MqttVersion(int numericValue) {
+            this.numericValue = numericValue;
+        }
+
+        public int getNumericValue() {
+            return numericValue;
+        }
+    }
+
+    public enum SupportedSchemes {
+        TCP,
+        SSL,
+        WS,
+        WSS;
+
+        public static String getValuesAsString(String delimiter) {
+            return String.join(delimiter, Arrays.stream(values()).map(value -> value.name().toLowerCase()).toArray(String[]::new));
+        }
+    }

Review Comment:
   Changed and renamed based on the review of @turcsanyip .



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957859728


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -384,16 +326,59 @@ public final void onTrigger(final ProcessContext context, final ProcessSessionFa
             onTrigger(context, session);
             session.commitAsync();
         } catch (final Throwable t) {
-            getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, t});
+            getLogger().error("{} failed to process due to {}; rolling back session", this, t);
             session.rollback(true);
             throw t;
         }
     }
 
     public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
 
-    protected boolean isConnected(){
+    protected boolean isConnected() {
         return (mqttClient != null && mqttClient.isConnected());
     }
 
+    protected MqttClientProperties getMqttClientProperties(final ProcessContext context) {
+        final MqttClientProperties clientProperties = new MqttClientProperties();
+
+        try {
+            clientProperties.setBrokerUri(new URI(context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue()));
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+
+        String clientId = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
+        if (clientId == null) {
+            clientId = UUID.randomUUID().toString();
+        }
+        clientProperties.setClientId(clientId);
+
+        clientProperties.setMqttVersion(MqttVersion.fromVersionCode(context.getProperty(PROP_MQTT_VERSION).asInteger()));
+
+        clientProperties.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
+        clientProperties.setSessionExpiryInterval(context.getProperty(PROP_SESSION_EXPIRY_INTERVAL).asTimePeriod(TimeUnit.SECONDS));
+
+        clientProperties.setKeepAliveInterval(context.getProperty(PROP_KEEP_ALIVE_INTERVAL).asInteger());
+        clientProperties.setConnectionTimeout(context.getProperty(PROP_CONN_TIMEOUT).asInteger());
+
+        final PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
+        if (sslProp.isSet()) {
+            clientProperties.setSslContextService((SSLContextService) sslProp.asControllerService());

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957824765


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();
             }
-            return new ValidationResult.Builder().subject(subject).valid(true).build();
+            if (!isValidEnumIgnoreCase(MqttProtocolScheme.class, brokerURI.getScheme())) {
+                return new ValidationResult.Builder().subject(subject).valid(false)
+                        .explanation("invalid scheme! supported schemes are: " + MqttProtocolScheme.getValuesAsString(", ")).build();

Review Comment:
   Double colon feels weird in the same sentence. What about dot? The message this way will be `'Broker URI' is invalid because scheme is invalid. Supported schemes are: tcp, ssl, ws, wss` Invalid is repeated but still better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r945584486


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);

Review Comment:
   I moved this catch to ConsumeMQTT, this way, the same exception will be thrown for both clients. Is it ok that way?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941398697


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);
+                    }
+                })
+                .send();
+    }
+
+    @Override
+    public void setCallback(NifiMqttCallback callback) {
+        this.callback = callback;
+    }
+
+    public static KeyManagerFactory getKeyManagerFactory(String keyStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore keyStore = loadIntoKeyStore(keyStoreType, path, keyStorePassword);
+
+            final KeyManagerFactory kmf = KeyManagerFactory
+                    .getInstance(KeyManagerFactory.getDefaultAlgorithm());
+            kmf.init(keyStore, new char[0]); // https://stackoverflow.com/questions/1814048/sun-java-keymanagerfactory-and-null-passwords
+
+            return kmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static TrustManagerFactory getTrustManagerFactory(String trustStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore trustStore = loadIntoKeyStore(trustStoreType, path, keyStorePassword);
+
+            final TrustManagerFactory tmf = TrustManagerFactory
+                    .getInstance(TrustManagerFactory.getDefaultAlgorithm());
+            tmf.init(trustStore);
+
+            return tmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static KeyStore loadIntoKeyStore(String type, String path, char[] keyStorePassword) throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+        final KeyStore keyStore = KeyStore.getInstance(type);
+        final InputStream in = new FileInputStream(path);
+        keyStore.load(in, keyStorePassword);
+        return keyStore;
+    }

Review Comment:
   It looks like these methods could be replaced with methods from `KeyStoreUtils` in `nifi-security-utils`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941410795


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {

Review Comment:
   Since `NifiMqttException` is a subclass of `RuntimeException`, it does not need to be declared.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on PR #6225:
URL: https://github.com/apache/nifi/pull/6225#issuecomment-1224352753

   Thank you for the review @turcsanyip, @tpalfy and @exceptionfactory! I've tried to address your comments!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957858775


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();

Review Comment:
   Thanks for noticing it. Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957858106


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttClientProperties.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ssl.SSLContextService;
+
+import java.net.URI;
+
+public class MqttClientProperties {
+    private URI brokerUri;
+    private String clientId;
+
+    private MqttVersion mqttVersion;
+
+    private int keepAliveInterval;
+    private int connectionTimeout;
+
+    private boolean cleanSession;
+    private Long sessionExpiryInterval;
+
+    private SSLContextService sslContextService;

Review Comment:
   Thank you for the idea! Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957500825


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();
             }
-            return new ValidationResult.Builder().subject(subject).valid(true).build();
+            if (!isValidEnumIgnoreCase(MqttProtocolScheme.class, brokerURI.getScheme())) {
+                return new ValidationResult.Builder().subject(subject).valid(false)
+                        .explanation("invalid scheme! supported schemes are: " + MqttProtocolScheme.getValuesAsString(", ")).build();

Review Comment:
   Error messages should not use the exclamation mark character.
   ```suggestion
                           .explanation("Invalid Scheme: supported schemes are: " + MqttProtocolScheme.getValuesAsString(", ")).build();
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java:
##########
@@ -649,38 +625,44 @@ private void closeWriter(final RecordSetWriter writer) {
     }
 
     private String getTransitUri(String... appends) {
-        StringBuilder stringBuilder = new StringBuilder(brokerUri);
-        for(String append : appends) {
+        String broker = clientProperties.getBrokerUri().toString();
+        StringBuilder stringBuilder = new StringBuilder(broker.endsWith("/") ? broker : broker + "/");
+        for (String append : appends) {
             stringBuilder.append(append);
         }
         return stringBuilder.toString();
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);

Review Comment:
   ```suggestion
           logger.error("Connection to {} lost due to: {}", clientProperties.getBroker(), cause.getMessage(), cause);
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java:
##########
@@ -201,35 +191,35 @@ private void initializeClient(ProcessContext context) {
         // non-null but not connected, so we need to handle each case and only create a new client when it is null
         try {
             if (mqttClient == null) {
-                logger.debug("Creating client");
-                mqttClient = createMqttClient(broker, clientID, persistence);
+                mqttClient = createMqttClient();
                 mqttClient.setCallback(this);
             }
 
             if (!mqttClient.isConnected()) {
-                logger.debug("Connecting client");
-                mqttClient.connect(connOpts);
+                mqttClient.connect();
             }
-        } catch (MqttException e) {
-            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
+        } catch (Exception e) {
+            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", clientProperties.getBroker(), e);
             context.yield();
         }
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);

Review Comment:
   The object array wrapper can be removed.
   ```suggestion
           logger.error("Connection to {} lost due to: {}", clientProperties.getBroker(), cause.getMessage(), cause);
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttProtocolScheme.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 java.util.Arrays;
+
+public enum MqttProtocolScheme {
+    TCP,
+    SSL,
+    WS,
+    WSS;
+
+    public static String getValuesAsString(String delimiter) {
+        return String.join(delimiter, Arrays.stream(values()).map(value -> value.name().toLowerCase()).toArray(String[]::new));
+    }

Review Comment:
   It looks like this method is used only in reference to an error message. Recommend removing it from the `enum` and creating a local private method instead.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java:
##########
@@ -649,38 +625,44 @@ private void closeWriter(final RecordSetWriter writer) {
     }
 
     private String getTransitUri(String... appends) {
-        StringBuilder stringBuilder = new StringBuilder(brokerUri);
-        for(String append : appends) {
+        String broker = clientProperties.getBrokerUri().toString();
+        StringBuilder stringBuilder = new StringBuilder(broker.endsWith("/") ? broker : broker + "/");
+        for (String append : appends) {
             stringBuilder.append(append);
         }
         return stringBuilder.toString();
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);
     }
 
     @Override
-    public void messageArrived(String topic, MqttMessage message) throws Exception {
+    public void messageArrived(ReceivedMqttMessage message) {
         if (logger.isDebugEnabled()) {
             byte[] payload = message.getPayload();
-            String text = new String(payload, "UTF-8");
+            String text = new String(payload, StandardCharsets.UTF_8);
             if (StringUtils.isAsciiPrintable(text)) {
-                logger.debug("Message arrived from topic {}. Payload: {}", new Object[] {topic, text});
+                logger.debug("Message arrived from topic {}. Payload: {}", message.getTopic(), text);
             } else {
-                logger.debug("Message arrived from topic {}. Binary value of size {}", new Object[] {topic, payload.length});
+                logger.debug("Message arrived from topic {}. Binary value of size {}", message.getTopic(), payload.length);
             }
         }
 
-        if(!mqttQueue.offer(new MQTTQueueMessage(topic, message), 1, TimeUnit.SECONDS)) {
-            throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
+        try {
+            if (!mqttQueue.offer(message, 1, TimeUnit.SECONDS)) {
+                throw new IllegalStateException("The subscriber queue is full, cannot receive another message until the processor is scheduled to run.");
+            }
+        } catch (InterruptedException e) {
+            throw new MqttException("Failed to process message arrived from topic " + message.getTopic());
         }
     }
 
     @Override
-    public void deliveryComplete(IMqttDeliveryToken token) {
-        logger.warn("Received MQTT 'delivery complete' message to subscriber: " + token);
+    public void deliveryComplete(String token) {
+        // Unlikely situation. Api uses the same callback for publisher and consumer as well.
+        // That's why we have this log message here to indicate something really messy thing happened.
+        logger.error("Received MQTT 'delivery complete' message to subscriber: " + token);

Review Comment:
   ```suggestion
           logger.error("Received MQTT 'delivery complete' message to subscriber token [{}]", token);
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -384,16 +326,59 @@ public final void onTrigger(final ProcessContext context, final ProcessSessionFa
             onTrigger(context, session);
             session.commitAsync();
         } catch (final Throwable t) {
-            getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, t});
+            getLogger().error("{} failed to process due to {}; rolling back session", this, t);
             session.rollback(true);
             throw t;
         }
     }
 
     public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
 
-    protected boolean isConnected(){
+    protected boolean isConnected() {
         return (mqttClient != null && mqttClient.isConnected());
     }
 
+    protected MqttClientProperties getMqttClientProperties(final ProcessContext context) {
+        final MqttClientProperties clientProperties = new MqttClientProperties();
+
+        try {
+            clientProperties.setBrokerUri(new URI(context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue()));
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);

Review Comment:
   The error should be changed to an `IllegalArgumentException` and should include a message.
   ```suggestion
               throw new IllegalArgumentException("Invalid Broker URI", e);
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -289,88 +285,34 @@ public Collection<ValidationResult> customValidate(final ValidationContext valid
         return results;
     }
 
-    public static Properties transformSSLContextService(SSLContextService sslContextService){
-        Properties properties = new Properties();
-        if (sslContextService.getSslAlgorithm() != null) {
-            properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
-        }
-        if (sslContextService.getKeyStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.keyStore", sslContextService.getKeyStoreFile());
-        }
-        if (sslContextService.getKeyStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.keyStorePassword", sslContextService.getKeyStorePassword());
-        }
-        if (sslContextService.getKeyStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.keyStoreType", sslContextService.getKeyStoreType());
-        }
-        if (sslContextService.getTrustStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.trustStore", sslContextService.getTrustStoreFile());
-        }
-        if (sslContextService.getTrustStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.trustStorePassword", sslContextService.getTrustStorePassword());
-        }
-        if (sslContextService.getTrustStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.trustStoreType", sslContextService.getTrustStoreType());
-        }
-        return  properties;
+    protected void onScheduled(final ProcessContext context) {
+        clientProperties = getMqttClientProperties(context);
     }
 
-    protected void onScheduled(final ProcessContext context){
-        broker = context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue();
-        brokerUri = broker.endsWith("/") ? broker : broker + "/";
-        clientID = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
-
-        if (clientID == null) {
-            clientID = UUID.randomUUID().toString();
-        }
-
-        connOpts = new MqttConnectOptions();
-        connOpts.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
-        connOpts.setKeepAliveInterval(context.getProperty(PROP_KEEP_ALIVE_INTERVAL).asInteger());
-        connOpts.setMqttVersion(context.getProperty(PROP_MQTT_VERSION).asInteger());
-        connOpts.setConnectionTimeout(context.getProperty(PROP_CONN_TIMEOUT).asInteger());
-
-        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()){
-            String lastWillMessage = context.getProperty(PROP_LAST_WILL_MESSAGE).getValue();
-            PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_RETAIN);
-            Integer lastWillQOS = context.getProperty(PROP_LAST_WILL_QOS).asInteger();
-            connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getBytes(), lastWillQOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
-        }
-
-
-        PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
-        if(usernameProp.isSet()) {
-            connOpts.setUserName(usernameProp.evaluateAttributeExpressions().getValue());
-            connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
-        }
-    }
+    protected void stopClient() {
+        // Since client is created in the onTrigger method it can happen that it never will be created because of an initialization error.
+        // We are preventing additional nullPtrException here, but the clean solution would be to create the client in the onScheduled method.
+        if (mqttClient != null) {
+            try {
+                logger.info("Disconnecting client");
+                mqttClient.disconnect();
+            } catch (Exception e) {
+                logger.error("Error disconnecting MQTT client due to {}", new Object[]{e.getMessage()}, e);
+            }
 
-    protected void onStopped() {
-        try {
-            logger.info("Disconnecting client");
-            mqttClient.disconnect(DISCONNECT_TIMEOUT);
-        } catch(MqttException me) {
-            logger.error("Error disconnecting MQTT client due to {}", new Object[]{me.getMessage()}, me);
-        }
+            try {
+                logger.info("Closing client");
+                mqttClient.close();
+            } catch (Exception e) {
+                logger.error("Error closing MQTT client due to {}", new Object[]{e.getMessage()}, e);

Review Comment:
   The message could be removed along with the Object array wrapper, since the stack trace includes the message.
   ```suggestion
                   logger.error("Error closing MQTT client", e);
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5ClientBuilder;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import com.hivemq.client.mqtt.mqtt5.message.subscribe.suback.Mqtt5SubAck;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.mqtt.common.MqttCallback;
+import org.apache.nifi.processors.mqtt.common.MqttClient;
+import org.apache.nifi.processors.mqtt.common.MqttClientProperties;
+import org.apache.nifi.processors.mqtt.common.MqttException;
+import org.apache.nifi.processors.mqtt.common.ReceivedMqttMessage;
+import org.apache.nifi.processors.mqtt.common.StandardMqttMessage;
+import org.apache.nifi.security.util.KeyStoreUtils;
+import org.apache.nifi.security.util.TlsException;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.processors.mqtt.common.MqttProtocolScheme.SSL;
+import static org.apache.nifi.processors.mqtt.common.MqttProtocolScheme.WS;
+import static org.apache.nifi.processors.mqtt.common.MqttProtocolScheme.WSS;
+
+public class HiveMqV5ClientAdapter implements MqttClient {
+
+    private final Mqtt5BlockingClient mqtt5BlockingClient;
+    private final MqttClientProperties clientProperties;
+    private final ComponentLog logger;
+
+    private MqttCallback callback;
+
+    public HiveMqV5ClientAdapter(MqttClientProperties clientProperties, ComponentLog logger) throws TlsException {
+        this.mqtt5BlockingClient = createClient(clientProperties, logger);
+        this.clientProperties = clientProperties;
+        this.logger = logger;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5BlockingClient.getState().isConnected();
+    }
+
+    @Override
+    public void connect() {
+        logger.debug("Connecting to broker");
+
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(clientProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = clientProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(clientProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = clientProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(clientProperties.getLastWillMessage().getBytes())
+                    .retain(clientProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(clientProperties.getLastWillQos()))
+                    .applyWillPublish();
+        }
+
+        final String username = clientProperties.getUsername();
+        final String password = clientProperties.getPassword();
+        if (username != null && password != null) {
+            connectBuilder.simpleAuth()
+                    .username(clientProperties.getUsername())
+                    .password(password.getBytes(StandardCharsets.UTF_8))
+                    .applySimpleAuth();
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5BlockingClient.connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect() {
+        logger.debug("Disconnecting client");
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client.
+        mqtt5BlockingClient.disconnect();
+    }
+
+    @Override
+    public void close() {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, StandardMqttMessage message) {
+        logger.debug("Publishing message to {} with QoS: {}", topic, message.getQos());
+
+        mqtt5BlockingClient.publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        logger.debug("Subscribing to {} with QoS: {}", topicFilter, qos);
+
+        CompletableFuture<Mqtt5SubAck> futureAck = mqtt5BlockingClient.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final ReceivedMqttMessage receivedMessage = new ReceivedMqttMessage(
+                            mqtt5Publish.getPayloadAsBytes(),
+                            mqtt5Publish.getQos().getCode(),
+                            mqtt5Publish.isRetain(),
+                            mqtt5Publish.getTopic().toString());
+                    callback.messageArrived(receivedMessage);
+                })
+                .send();
+
+        // Setting "listener" callback is only possible with async client, though sending subscribe message
+        // should happen in a blocking way to make sure the processor is blocked until ack is not arrived.
+        try {
+            Mqtt5SubAck ack = futureAck.get(clientProperties.getConnectionTimeout(), TimeUnit.SECONDS);
+            logger.debug("Received mqtt5 subscribe ack: {}", ack.toString());

Review Comment:
   The `toString()` call is not necessary.
   ```suggestion
               logger.debug("Received mqtt5 subscribe ack: {}", ack);
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttClientProperties.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ssl.SSLContextService;
+
+import java.net.URI;
+
+public class MqttClientProperties {
+    private URI brokerUri;
+    private String clientId;
+
+    private MqttVersion mqttVersion;
+
+    private int keepAliveInterval;
+    private int connectionTimeout;
+
+    private boolean cleanSession;
+    private Long sessionExpiryInterval;
+
+    private SSLContextService sslContextService;

Review Comment:
   Instead of passing the `SSLContextService` reference, the `TlsConfiguration` object should be used as it contains all of the necessary properties.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();

Review Comment:
   There should be a space before the path.
   ```suggestion
                   return new ValidationResult.Builder().subject(subject).valid(false).explanation("the broker URI cannot have a path. It currently is: " + brokerURI.getPath()).build();
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -384,16 +326,59 @@ public final void onTrigger(final ProcessContext context, final ProcessSessionFa
             onTrigger(context, session);
             session.commitAsync();
         } catch (final Throwable t) {
-            getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, t});
+            getLogger().error("{} failed to process due to {}; rolling back session", this, t);
             session.rollback(true);
             throw t;
         }
     }
 
     public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
 
-    protected boolean isConnected(){
+    protected boolean isConnected() {
         return (mqttClient != null && mqttClient.isConnected());
     }
 
+    protected MqttClientProperties getMqttClientProperties(final ProcessContext context) {
+        final MqttClientProperties clientProperties = new MqttClientProperties();
+
+        try {
+            clientProperties.setBrokerUri(new URI(context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue()));
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+
+        String clientId = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
+        if (clientId == null) {
+            clientId = UUID.randomUUID().toString();
+        }
+        clientProperties.setClientId(clientId);
+
+        clientProperties.setMqttVersion(MqttVersion.fromVersionCode(context.getProperty(PROP_MQTT_VERSION).asInteger()));
+
+        clientProperties.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
+        clientProperties.setSessionExpiryInterval(context.getProperty(PROP_SESSION_EXPIRY_INTERVAL).asTimePeriod(TimeUnit.SECONDS));
+
+        clientProperties.setKeepAliveInterval(context.getProperty(PROP_KEEP_ALIVE_INTERVAL).asInteger());
+        clientProperties.setConnectionTimeout(context.getProperty(PROP_CONN_TIMEOUT).asInteger());
+
+        final PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
+        if (sslProp.isSet()) {
+            clientProperties.setSslContextService((SSLContextService) sslProp.asControllerService());

Review Comment:
   As mentioned in the Client Properties, this can be adjusted to call `createTlsConfiguration()` to pass the properties.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java:
##########
@@ -201,35 +191,35 @@ private void initializeClient(ProcessContext context) {
         // non-null but not connected, so we need to handle each case and only create a new client when it is null
         try {
             if (mqttClient == null) {
-                logger.debug("Creating client");
-                mqttClient = createMqttClient(broker, clientID, persistence);
+                mqttClient = createMqttClient();
                 mqttClient.setCallback(this);
             }
 
             if (!mqttClient.isConnected()) {
-                logger.debug("Connecting client");
-                mqttClient.connect(connOpts);
+                mqttClient.connect();
             }
-        } catch (MqttException e) {
-            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
+        } catch (Exception e) {
+            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", clientProperties.getBroker(), e);
             context.yield();
         }
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);
     }
 
     @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())+"}");
+    public void messageArrived(ReceivedMqttMessage message) {
+        // Unlikely situation. Api uses the same callback for publisher and consumer as well.
+        // That's why we have this log message here to indicate something really messy thing happened.
+        logger.error("Message arrived to a PublishMQTT processor { topic:'" + message.getTopic() + "; payload:" + Arrays.toString(message.getPayload()) + "}");
     }
 
     @Override
-    public void deliveryComplete(IMqttDeliveryToken token) {
+    public void deliveryComplete(String 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.trace("Received 'delivery complete' message from broker for:" + token.toString());
+        logger.trace("Received 'delivery complete' message from broker for:" + token);

Review Comment:
   This should be changed to use a placeholder variable instead of concatenation.
   ```suggestion
           logger.trace("Received 'delivery complete' message from broker token [{}]", token);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957965217


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,94 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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.Arrays;
 import java.util.Collection;
 import java.util.List;
-import java.util.Properties;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();
             }
-            return new ValidationResult.Builder().subject(subject).valid(true).build();
+            if (!isValidEnumIgnoreCase(MqttProtocolScheme.class, brokerURI.getScheme())) {
+                return new ValidationResult.Builder().subject(subject).valid(false)
+                        .explanation("scheme is invalid. Supported schemes are: " + getSupportedSchemeList()).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);
-            }
+    private static String getSupportedSchemeList() {
+        return String.join(", ", Arrays.stream(MqttProtocolScheme.values()).map(value -> value.name().toLowerCase()).toArray(String[]::new));

Review Comment:
   This can be modified to use Collectors.joining():
   ```suggestion
           return Arrays.stream(MqttProtocolScheme.values())
               .map(value -> value.name().toLowerCase())
               .collect(Collectors.joining(", ");
   ```



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -384,16 +331,60 @@ public final void onTrigger(final ProcessContext context, final ProcessSessionFa
             onTrigger(context, session);
             session.commitAsync();
         } catch (final Throwable t) {
-            getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, t});
+            getLogger().error("{} failed to process due to {}; rolling back session", this, t);
             session.rollback(true);
             throw t;
         }
     }
 
     public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
 
-    protected boolean isConnected(){
+    protected boolean isConnected() {
         return (mqttClient != null && mqttClient.isConnected());
     }
 
+    protected MqttClientProperties getMqttClientProperties(final ProcessContext context) {
+        final MqttClientProperties clientProperties = new MqttClientProperties();
+
+        try {
+            clientProperties.setBrokerUri(new URI(context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue()));
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("Invalid Broker URI", e);
+        }
+
+        String clientId = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
+        if (clientId == null) {
+            clientId = UUID.randomUUID().toString();
+        }
+        clientProperties.setClientId(clientId);
+
+        clientProperties.setMqttVersion(MqttVersion.fromVersionCode(context.getProperty(PROP_MQTT_VERSION).asInteger()));
+
+        clientProperties.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
+        clientProperties.setSessionExpiryInterval(context.getProperty(PROP_SESSION_EXPIRY_INTERVAL).asTimePeriod(TimeUnit.SECONDS));
+
+        clientProperties.setKeepAliveInterval(context.getProperty(PROP_KEEP_ALIVE_INTERVAL).asInteger());
+        clientProperties.setConnectionTimeout(context.getProperty(PROP_CONN_TIMEOUT).asInteger());
+
+        final PropertyValue sslProp = context.getProperty(PROP_SSL_CONTEXT_SERVICE);
+        if (sslProp.isSet()) {
+            final SSLContextService sslContextService = (SSLContextService) sslProp.asControllerService();

Review Comment:
   This should be changed to pass the reference class to `asControllerService()`:
   ```suggestion
               final SSLContextService sslContextService = sslProp.asControllerService(SSLContextService.class);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] asfgit closed pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors
URL: https://github.com/apache/nifi/pull/6225


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952881717


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);

Review Comment:
   Changed.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);
+
+        private final int numericValue;

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r953196811


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -289,88 +288,35 @@ public Collection<ValidationResult> customValidate(final ValidationContext valid
         return results;
     }
 
-    public static Properties transformSSLContextService(SSLContextService sslContextService){
-        Properties properties = new Properties();
-        if (sslContextService.getSslAlgorithm() != null) {
-            properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
-        }
-        if (sslContextService.getKeyStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.keyStore", sslContextService.getKeyStoreFile());
-        }
-        if (sslContextService.getKeyStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.keyStorePassword", sslContextService.getKeyStorePassword());
-        }
-        if (sslContextService.getKeyStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.keyStoreType", sslContextService.getKeyStoreType());
-        }
-        if (sslContextService.getTrustStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.trustStore", sslContextService.getTrustStoreFile());
-        }
-        if (sslContextService.getTrustStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.trustStorePassword", sslContextService.getTrustStorePassword());
-        }
-        if (sslContextService.getTrustStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.trustStoreType", sslContextService.getTrustStoreType());
-        }
-        return  properties;
-    }
-
-    protected void onScheduled(final ProcessContext context){
-        broker = context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue();
-        brokerUri = broker.endsWith("/") ? broker : broker + "/";
-        clientID = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
-
-        if (clientID == null) {
-            clientID = UUID.randomUUID().toString();
-        }
-
-        connOpts = new MqttConnectOptions();
-        connOpts.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
-        connOpts.setKeepAliveInterval(context.getProperty(PROP_KEEP_ALIVE_INTERVAL).asInteger());
-        connOpts.setMqttVersion(context.getProperty(PROP_MQTT_VERSION).asInteger());
-        connOpts.setConnectionTimeout(context.getProperty(PROP_CONN_TIMEOUT).asInteger());
-
-        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()){
-            String lastWillMessage = context.getProperty(PROP_LAST_WILL_MESSAGE).getValue();
-            PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_RETAIN);
-            Integer lastWillQOS = context.getProperty(PROP_LAST_WILL_QOS).asInteger();
-            connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getBytes(), lastWillQOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
-        }
-
-
-        PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
-        if(usernameProp.isSet()) {
-            connOpts.setUserName(usernameProp.evaluateAttributeExpressions().getValue());
-            connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
-        }
+    protected void onScheduled(final ProcessContext context) {
+        clientProperties = getMqttClientProperties(context);
+        connectionProperties = getMqttConnectionProperties(context);
     }
 
     protected void onStopped() {
-        try {
-            logger.info("Disconnecting client");
-            mqttClient.disconnect(DISCONNECT_TIMEOUT);
-        } catch(MqttException me) {
-            logger.error("Error disconnecting MQTT client due to {}", new Object[]{me.getMessage()}, me);
-        }
+        // Since client is created in the onTrigger method it can happen that it never will be created because of an initialization error.
+        // We are preventing additional nullPtrException here, but the clean solution would be to create the client in the onScheduled method.
+        if (mqttClient != null) {
+            try {
+                logger.info("Disconnecting client");
+                mqttClient.disconnect(DISCONNECT_TIMEOUT);
+            } catch (MqttException me) {
+                logger.error("Error disconnecting MQTT client due to {}", new Object[]{me.getMessage()}, me);
+            }
+
+            try {
+                logger.info("Closing client");
+                mqttClient.close();
+            } catch (MqttException me) {
+                logger.error("Error closing MQTT client due to {}", new Object[]{me.getMessage()}, me);
+            }

Review Comment:
   Yes, I've tested with v3 then switching to v5 and it worked because Paho adapter rethrows any exception as MqttException. But this is not true for v5. Thanks for noticing it. Changed!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941205695


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-nar/src/main/resources/META-INF/NOTICE:
##########
@@ -41,6 +41,11 @@ The following binary components are provided under the Apache Software License v
       in some artifacts (usually source distributions); but is always available
       from the source code management (SCM) system project uses.
 
+  (ASLv2) HiveMQ MQTT Client

Review Comment:
   Thanks for NOTICING that! (pun intended, change added)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941321632


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -199,7 +199,7 @@ public abstract class AbstractMQTTProcessor extends AbstractSessionFactoryProces
             .build();
 
     public static final PropertyDescriptor PROP_SESSION_EXPIRY_INTERVAL = new PropertyDescriptor.Builder()
-            .name("Session Expiry Interval")
+            .name("Session Expiry Interval (seconds)")

Review Comment:
   Reading more closely, I see this is a new property, so disregard the previous comment. Even though the unit of measure is different, I tend to agree with @tpalfy that using the Time Unit Validator is better for this new property.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941407304


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);
+
+        private final int numericValue;
+
+        MqttVersion(int numericValue) {
+            this.numericValue = numericValue;
+        }
+
+        public int getNumericValue() {
+            return numericValue;
+        }
+    }
+
+    public enum SupportedSchemes {
+        TCP,
+        SSL,
+        WS,
+        WSS;
+
+        public static String getValuesAsString(String delimiter) {
+            return String.join(delimiter, Arrays.stream(values()).map(value -> value.name().toLowerCase()).toArray(String[]::new));
+        }
+    }

Review Comment:
   Recommend promoting this to a separate file instead of embedded within `MqttConstants`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957813444


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java:
##########
@@ -201,35 +191,35 @@ private void initializeClient(ProcessContext context) {
         // non-null but not connected, so we need to handle each case and only create a new client when it is null
         try {
             if (mqttClient == null) {
-                logger.debug("Creating client");
-                mqttClient = createMqttClient(broker, clientID, persistence);
+                mqttClient = createMqttClient();
                 mqttClient.setCallback(this);
             }
 
             if (!mqttClient.isConnected()) {
-                logger.debug("Connecting client");
-                mqttClient.connect(connOpts);
+                mqttClient.connect();
             }
-        } catch (MqttException e) {
-            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
+        } catch (Exception e) {
+            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", clientProperties.getBroker(), e);
             context.yield();
         }
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);

Review Comment:
   Are you sure? If I remove the object wrapper `void error(String msg, Object... os);` will be used instead of `void error(String msg, Object[] os, Throwable t);`. I didn't test what happens in this situation with NiFi logging, but if I remember right Slf4j throws an error when there are less placeholder than passed parameters. I expect the same in our case. Also we would loose the cause stack trace.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957824765


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();
             }
-            return new ValidationResult.Builder().subject(subject).valid(true).build();
+            if (!isValidEnumIgnoreCase(MqttProtocolScheme.class, brokerURI.getScheme())) {
+                return new ValidationResult.Builder().subject(subject).valid(false)
+                        .explanation("invalid scheme! supported schemes are: " + MqttProtocolScheme.getValuesAsString(", ")).build();

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957863215


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -289,88 +285,34 @@ public Collection<ValidationResult> customValidate(final ValidationContext valid
         return results;
     }
 
-    public static Properties transformSSLContextService(SSLContextService sslContextService){
-        Properties properties = new Properties();
-        if (sslContextService.getSslAlgorithm() != null) {
-            properties.setProperty("com.ibm.ssl.protocol", sslContextService.getSslAlgorithm());
-        }
-        if (sslContextService.getKeyStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.keyStore", sslContextService.getKeyStoreFile());
-        }
-        if (sslContextService.getKeyStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.keyStorePassword", sslContextService.getKeyStorePassword());
-        }
-        if (sslContextService.getKeyStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.keyStoreType", sslContextService.getKeyStoreType());
-        }
-        if (sslContextService.getTrustStoreFile() != null) {
-            properties.setProperty("com.ibm.ssl.trustStore", sslContextService.getTrustStoreFile());
-        }
-        if (sslContextService.getTrustStorePassword() != null) {
-            properties.setProperty("com.ibm.ssl.trustStorePassword", sslContextService.getTrustStorePassword());
-        }
-        if (sslContextService.getTrustStoreType() != null) {
-            properties.setProperty("com.ibm.ssl.trustStoreType", sslContextService.getTrustStoreType());
-        }
-        return  properties;
+    protected void onScheduled(final ProcessContext context) {
+        clientProperties = getMqttClientProperties(context);
     }
 
-    protected void onScheduled(final ProcessContext context){
-        broker = context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue();
-        brokerUri = broker.endsWith("/") ? broker : broker + "/";
-        clientID = context.getProperty(PROP_CLIENTID).evaluateAttributeExpressions().getValue();
-
-        if (clientID == null) {
-            clientID = UUID.randomUUID().toString();
-        }
-
-        connOpts = new MqttConnectOptions();
-        connOpts.setCleanSession(context.getProperty(PROP_CLEAN_SESSION).asBoolean());
-        connOpts.setKeepAliveInterval(context.getProperty(PROP_KEEP_ALIVE_INTERVAL).asInteger());
-        connOpts.setMqttVersion(context.getProperty(PROP_MQTT_VERSION).asInteger());
-        connOpts.setConnectionTimeout(context.getProperty(PROP_CONN_TIMEOUT).asInteger());
-
-        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()){
-            String lastWillMessage = context.getProperty(PROP_LAST_WILL_MESSAGE).getValue();
-            PropertyValue lastWillRetain = context.getProperty(PROP_LAST_WILL_RETAIN);
-            Integer lastWillQOS = context.getProperty(PROP_LAST_WILL_QOS).asInteger();
-            connOpts.setWill(lastWillTopicProp.getValue(), lastWillMessage.getBytes(), lastWillQOS, lastWillRetain.isSet() ? lastWillRetain.asBoolean() : false);
-        }
-
-
-        PropertyValue usernameProp = context.getProperty(PROP_USERNAME);
-        if(usernameProp.isSet()) {
-            connOpts.setUserName(usernameProp.evaluateAttributeExpressions().getValue());
-            connOpts.setPassword(context.getProperty(PROP_PASSWORD).getValue().toCharArray());
-        }
-    }
+    protected void stopClient() {
+        // Since client is created in the onTrigger method it can happen that it never will be created because of an initialization error.
+        // We are preventing additional nullPtrException here, but the clean solution would be to create the client in the onScheduled method.
+        if (mqttClient != null) {
+            try {
+                logger.info("Disconnecting client");
+                mqttClient.disconnect();
+            } catch (Exception e) {
+                logger.error("Error disconnecting MQTT client due to {}", new Object[]{e.getMessage()}, e);
+            }
 
-    protected void onStopped() {
-        try {
-            logger.info("Disconnecting client");
-            mqttClient.disconnect(DISCONNECT_TIMEOUT);
-        } catch(MqttException me) {
-            logger.error("Error disconnecting MQTT client due to {}", new Object[]{me.getMessage()}, me);
-        }
+            try {
+                logger.info("Closing client");
+                mqttClient.close();
+            } catch (Exception e) {
+                logger.error("Error closing MQTT client due to {}", new Object[]{e.getMessage()}, e);

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957339014


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final long DEFAULT_SESSION_EXPIRY_INTERVAL_IN_SECONDS = 3600;

Review Comment:
   Just a little addition to my feels too long comment. If I set the session timeout to that long, I have to think about how these messages will be stored on the broker (thousands of messages / sec for 24 hours) and then how these messages will be processed along with the new messages. Will NiFi be able to keep up with that temporary increased load? Ofc on the other side there is the problem of loosing data... Changing to 24 hours as we agreed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952879664


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);
+                    }
+                })
+                .send();
+    }
+
+    @Override
+    public void setCallback(NifiMqttCallback callback) {
+        this.callback = callback;
+    }
+
+    public static KeyManagerFactory getKeyManagerFactory(String keyStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore keyStore = loadIntoKeyStore(keyStoreType, path, keyStorePassword);
+
+            final KeyManagerFactory kmf = KeyManagerFactory
+                    .getInstance(KeyManagerFactory.getDefaultAlgorithm());
+            kmf.init(keyStore, new char[0]); // https://stackoverflow.com/questions/1814048/sun-java-keymanagerfactory-and-null-passwords
+
+            return kmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static TrustManagerFactory getTrustManagerFactory(String trustStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore trustStore = loadIntoKeyStore(trustStoreType, path, keyStorePassword);
+
+            final TrustManagerFactory tmf = TrustManagerFactory
+                    .getInstance(TrustManagerFactory.getDefaultAlgorithm());
+            tmf.init(trustStore);
+
+            return tmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static KeyStore loadIntoKeyStore(String type, String path, char[] keyStorePassword) throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+        final KeyStore keyStore = KeyStore.getInstance(type);
+        final InputStream in = new FileInputStream(path);
+        keyStore.load(in, keyStorePassword);
+        return keyStore;
+    }

Review Comment:
   Thanks for the info, I didn't know about them! Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957824765


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -31,89 +31,89 @@
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.TlsException;
 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 java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.commons.lang3.EnumUtils.isValidEnumIgnoreCase;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
 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_500;
 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 {
 
-    public static int DISCONNECT_TIMEOUT = 5000;
+    private static final String DEFAULT_SESSION_EXPIRY_INTERVAL = "24 hrs";
 
     protected ComponentLog logger;
-    protected IMqttClient mqttClient;
-    protected volatile String broker;
-    protected volatile String brokerUri;
-    protected volatile String clientID;
-    protected MqttConnectOptions connOpts;
-    protected MemoryPersistence persistence = new MemoryPersistence();
 
-    public ProcessSessionFactory processSessionFactory;
+    protected MqttClientProperties clientProperties;
 
-    public static final Validator QOS_VALIDATOR = new Validator() {
+    protected MqttClientFactory mqttClientFactory = new MqttClientFactory();
+    protected MqttClient mqttClient;
 
-        @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 ProcessSessionFactory processSessionFactory;
+
+    public static final Validator QOS_VALIDATOR = (subject, input, 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()) || "ws".equals(brokerURI.getScheme()) || "wss".equals(brokerURI.getScheme()))) {
-                    return new ValidationResult.Builder().subject(subject).valid(false).explanation("only the 'tcp', 'ssl', 'ws' and 'wss' schemes are supported.").build();
-                }
-            } catch (URISyntaxException e) {
-                return new ValidationResult.Builder().subject(subject).valid(false).explanation("it is not valid URI syntax.").build();
+    public static final Validator BROKER_VALIDATOR = (subject, input, context) -> {
+        try {
+            URI brokerURI = new URI(input);
+            if (!EMPTY.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();
             }
-            return new ValidationResult.Builder().subject(subject).valid(true).build();
+            if (!isValidEnumIgnoreCase(MqttProtocolScheme.class, brokerURI.getScheme())) {
+                return new ValidationResult.Builder().subject(subject).valid(false)
+                        .explanation("invalid scheme! supported schemes are: " + MqttProtocolScheme.getValuesAsString(", ")).build();

Review Comment:
   Double colon feels weird in the same sentence. What about comma? Btw in this class all explanation starts with lowercase. What is the rule, which approach should I follow?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on PR #6225:
URL: https://github.com/apache/nifi/pull/6225#issuecomment-1231328126

   @exceptionfactory Thanks for the review! Your latest minor suggestions will be addressed in an upcoming PR ([NIFI-10411](https://issues.apache.org/jira/browse/NIFI-10411)) in order to avoid the extra build cycle here.
   
   Merging to main... 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] tpalfy commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941386857


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);
+                    }
+                })
+                .send();
+    }
+
+    @Override
+    public void setCallback(NifiMqttCallback callback) {
+        this.callback = callback;
+    }
+
+    public static KeyManagerFactory getKeyManagerFactory(String keyStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore keyStore = loadIntoKeyStore(keyStoreType, path, keyStorePassword);
+
+            final KeyManagerFactory kmf = KeyManagerFactory
+                    .getInstance(KeyManagerFactory.getDefaultAlgorithm());
+            kmf.init(keyStore, new char[0]); // https://stackoverflow.com/questions/1814048/sun-java-keymanagerfactory-and-null-passwords
+
+            return kmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static TrustManagerFactory getTrustManagerFactory(String trustStoreType, String path, char[] keyStorePassword) {
+        try {
+            final KeyStore trustStore = loadIntoKeyStore(trustStoreType, path, keyStorePassword);
+
+            final TrustManagerFactory tmf = TrustManagerFactory
+                    .getInstance(TrustManagerFactory.getDefaultAlgorithm());
+            tmf.init(trustStore);
+
+            return tmf;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static KeyStore loadIntoKeyStore(String type, String path, char[] keyStorePassword) throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+        final KeyStore keyStore = KeyStore.getInstance(type);
+        final InputStream in = new FileInputStream(path);
+        keyStore.load(in, keyStorePassword);
+        return keyStore;
+    }
+
+    private byte[] toBytes(char[] chars) {
+        final ByteBuffer byteBuffer = StandardCharsets.UTF_8.encode(CharBuffer.wrap(chars));
+        final byte[] bytes = Arrays.copyOfRange(byteBuffer.array(), byteBuffer.position(), byteBuffer.limit());
+        clearSensitive(byteBuffer.array());
+        return bytes;
+    }
+
+    private void clearSensitive(char[] chars) {

Review Comment:
   Clearing the password from the _MqttConnectionProperties_ prevents the the reconnection logic to execute successfully.
   I'd rather remove these _clear_ methods as acquiring the password when you have access to the memory of the nifi process is not preventable like this anyway.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941405400


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/NifiMqttClient.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+public interface NifiMqttClient {

Review Comment:
   In general it is better to avoid prefixing class names with `Nifi` since that is already implied through the package name. Perhaps something like `ProcessorMqttClient` or `ComponentMqttClient` would be better, open to other alternatives.
   
   As an interface, it would also be helpful to include JavaDocs with at least one sentence describing basic behavior.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941406015


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/NifiMqttMessage.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+public class NifiMqttMessage {

Review Comment:
   For this class `StandardMqttMessage` seems like a better name to avoid the `Nifi` prefix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r941410138


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);

Review Comment:
   Exceptions should include a basic message instead of just wrapping the cause. In this case, something like `Arrived message processing failed` would be helpful, include the `topicFilter` would also be useful for troubleshooting.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on PR #6225:
URL: https://github.com/apache/nifi/pull/6225#issuecomment-1225043867

   Rebased on top of current main, that's why the force push.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957814572


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/PublishMQTT.java:
##########
@@ -201,35 +191,35 @@ private void initializeClient(ProcessContext context) {
         // non-null but not connected, so we need to handle each case and only create a new client when it is null
         try {
             if (mqttClient == null) {
-                logger.debug("Creating client");
-                mqttClient = createMqttClient(broker, clientID, persistence);
+                mqttClient = createMqttClient();
                 mqttClient.setCallback(this);
             }
 
             if (!mqttClient.isConnected()) {
-                logger.debug("Connecting client");
-                mqttClient.connect(connOpts);
+                mqttClient.connect();
             }
-        } catch (MqttException e) {
-            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", new Object[]{broker}, e);
+        } catch (Exception e) {
+            logger.error("Connection to {} lost (or was never connected) and connection failed. Yielding processor", clientProperties.getBroker(), e);
             context.yield();
         }
     }
 
     @Override
     public void connectionLost(Throwable cause) {
-        logger.error("Connection to {} lost due to: {}", new Object[]{broker, cause.getMessage()}, cause);
+        logger.error("Connection to {} lost due to: {}", new Object[]{clientProperties.getBroker(), cause.getMessage()}, cause);

Review Comment:
   Oh, after checking the implementation of the logger I see that it checks whether the last argument is Throwable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on PR #6225:
URL: https://github.com/apache/nifi/pull/6225#issuecomment-1230980728

   Thank you for your additional review @exceptionfactory! Please see my latest commit!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957862534


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/AbstractMQTTProcessor.java:
##########
@@ -384,16 +326,59 @@ public final void onTrigger(final ProcessContext context, final ProcessSessionFa
             onTrigger(context, session);
             session.commitAsync();
         } catch (final Throwable t) {
-            getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, t});
+            getLogger().error("{} failed to process due to {}; rolling back session", this, t);
             session.rollback(true);
             throw t;
         }
     }
 
     public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
 
-    protected boolean isConnected(){
+    protected boolean isConnected() {
         return (mqttClient != null && mqttClient.isConnected());
     }
 
+    protected MqttClientProperties getMqttClientProperties(final ProcessContext context) {
+        final MqttClientProperties clientProperties = new MqttClientProperties();
+
+        try {
+            clientProperties.setBrokerUri(new URI(context.getProperty(PROP_BROKER_URI).evaluateAttributeExpressions().getValue()));
+        } catch (URISyntaxException e) {
+            throw new RuntimeException(e);

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957817646


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5ClientBuilder;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import com.hivemq.client.mqtt.mqtt5.message.subscribe.suback.Mqtt5SubAck;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.mqtt.common.MqttCallback;
+import org.apache.nifi.processors.mqtt.common.MqttClient;
+import org.apache.nifi.processors.mqtt.common.MqttClientProperties;
+import org.apache.nifi.processors.mqtt.common.MqttException;
+import org.apache.nifi.processors.mqtt.common.ReceivedMqttMessage;
+import org.apache.nifi.processors.mqtt.common.StandardMqttMessage;
+import org.apache.nifi.security.util.KeyStoreUtils;
+import org.apache.nifi.security.util.TlsException;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.processors.mqtt.common.MqttProtocolScheme.SSL;
+import static org.apache.nifi.processors.mqtt.common.MqttProtocolScheme.WS;
+import static org.apache.nifi.processors.mqtt.common.MqttProtocolScheme.WSS;
+
+public class HiveMqV5ClientAdapter implements MqttClient {
+
+    private final Mqtt5BlockingClient mqtt5BlockingClient;
+    private final MqttClientProperties clientProperties;
+    private final ComponentLog logger;
+
+    private MqttCallback callback;
+
+    public HiveMqV5ClientAdapter(MqttClientProperties clientProperties, ComponentLog logger) throws TlsException {
+        this.mqtt5BlockingClient = createClient(clientProperties, logger);
+        this.clientProperties = clientProperties;
+        this.logger = logger;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5BlockingClient.getState().isConnected();
+    }
+
+    @Override
+    public void connect() {
+        logger.debug("Connecting to broker");
+
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(clientProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = clientProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(clientProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = clientProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(clientProperties.getLastWillMessage().getBytes())
+                    .retain(clientProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(clientProperties.getLastWillQos()))
+                    .applyWillPublish();
+        }
+
+        final String username = clientProperties.getUsername();
+        final String password = clientProperties.getPassword();
+        if (username != null && password != null) {
+            connectBuilder.simpleAuth()
+                    .username(clientProperties.getUsername())
+                    .password(password.getBytes(StandardCharsets.UTF_8))
+                    .applySimpleAuth();
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5BlockingClient.connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect() {
+        logger.debug("Disconnecting client");
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client.
+        mqtt5BlockingClient.disconnect();
+    }
+
+    @Override
+    public void close() {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, StandardMqttMessage message) {
+        logger.debug("Publishing message to {} with QoS: {}", topic, message.getQos());
+
+        mqtt5BlockingClient.publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        logger.debug("Subscribing to {} with QoS: {}", topicFilter, qos);
+
+        CompletableFuture<Mqtt5SubAck> futureAck = mqtt5BlockingClient.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final ReceivedMqttMessage receivedMessage = new ReceivedMqttMessage(
+                            mqtt5Publish.getPayloadAsBytes(),
+                            mqtt5Publish.getQos().getCode(),
+                            mqtt5Publish.isRetain(),
+                            mqtt5Publish.getTopic().toString());
+                    callback.messageArrived(receivedMessage);
+                })
+                .send();
+
+        // Setting "listener" callback is only possible with async client, though sending subscribe message
+        // should happen in a blocking way to make sure the processor is blocked until ack is not arrived.
+        try {
+            Mqtt5SubAck ack = futureAck.get(clientProperties.getConnectionTimeout(), TimeUnit.SECONDS);
+            logger.debug("Received mqtt5 subscribe ack: {}", ack.toString());

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r957823553


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttProtocolScheme.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 java.util.Arrays;
+
+public enum MqttProtocolScheme {
+    TCP,
+    SSL,
+    WS,
+    WSS;
+
+    public static String getValuesAsString(String delimiter) {
+        return String.join(delimiter, Arrays.stream(values()).map(value -> value.name().toLowerCase()).toArray(String[]::new));
+    }

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952894224


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);

Review Comment:
   Changed.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.NifiMqttCallback;
+import org.apache.nifi.processors.mqtt.common.NifiMqttClient;
+import org.apache.nifi.processors.mqtt.common.NifiMqttException;
+import org.apache.nifi.processors.mqtt.common.NifiMqttMessage;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Objects;
+
+public class HiveMqV5ClientAdapter implements NifiMqttClient {
+
+    private final Mqtt5Client mqtt5Client;
+
+    private NifiMqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient) {
+        this.mqtt5Client = mqtt5BlockingClient;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5Client.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) throws NifiMqttException {
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        // checking for presence of password because username can be null
+        final char[] password = connectionProperties.getPassword();
+        if (password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(toBytes(password))
+                    .applySimpleAuth();
+
+            clearSensitive(connectionProperties.getPassword());
+            clearSensitive(password);
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5Client.toBlocking().connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) throws NifiMqttException {
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5Client.toBlocking().disconnect();
+    }
+
+    @Override
+    public void close() throws NifiMqttException {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, NifiMqttMessage message) throws NifiMqttException {
+        mqtt5Client.toAsync().publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        mqtt5Client.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final NifiMqttMessage nifiMqttMessage = new NifiMqttMessage();
+                    nifiMqttMessage.setPayload(mqtt5Publish.getPayloadAsBytes());
+                    nifiMqttMessage.setQos(mqtt5Publish.getQos().getCode());
+                    nifiMqttMessage.setRetained(mqtt5Publish.isRetain());
+                    try {
+                        callback.messageArrived(mqtt5Publish.getTopic().toString(), nifiMqttMessage);
+                    } catch (Exception e) {
+                        throw new NifiMqttException(e);
+                    }
+                })
+                .send();

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r953145681


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/adapters/HiveMqV5ClientAdapter.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.mqtt.adapters;
+
+import com.hivemq.client.mqtt.datatypes.MqttQos;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5BlockingClient;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5Connect;
+import com.hivemq.client.mqtt.mqtt5.message.connect.Mqtt5ConnectBuilder;
+import com.hivemq.client.mqtt.mqtt5.message.subscribe.suback.Mqtt5SubAck;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.mqtt.common.MqttCallback;
+import org.apache.nifi.processors.mqtt.common.MqttClient;
+import org.apache.nifi.processors.mqtt.common.MqttConnectionProperties;
+import org.apache.nifi.processors.mqtt.common.MqttException;
+import org.apache.nifi.processors.mqtt.common.ReceivedMqttMessage;
+import org.apache.nifi.processors.mqtt.common.StandardMqttMessage;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+
+public class HiveMqV5ClientAdapter implements MqttClient {
+
+    private final Mqtt5BlockingClient mqtt5BlockingClient;
+    private final ComponentLog logger;
+
+    private MqttCallback callback;
+
+    public HiveMqV5ClientAdapter(Mqtt5BlockingClient mqtt5BlockingClient, ComponentLog logger) {
+        this.mqtt5BlockingClient = mqtt5BlockingClient;
+        this.logger = logger;
+    }
+
+    @Override
+    public boolean isConnected() {
+        return mqtt5BlockingClient.getState().isConnected();
+    }
+
+    @Override
+    public void connect(MqttConnectionProperties connectionProperties) {
+        logger.debug("Connecting to broker");
+
+        final Mqtt5ConnectBuilder connectBuilder = Mqtt5Connect.builder()
+                .keepAlive(connectionProperties.getKeepAliveInterval());
+
+        final boolean cleanSession = connectionProperties.isCleanSession();
+        connectBuilder.cleanStart(cleanSession);
+        if (!cleanSession) {
+            connectBuilder.sessionExpiryInterval(connectionProperties.getSessionExpiryInterval());
+        }
+
+        final String lastWillTopic = connectionProperties.getLastWillTopic();
+        if (lastWillTopic != null) {
+            connectBuilder.willPublish()
+                    .topic(lastWillTopic)
+                    .payload(connectionProperties.getLastWillMessage().getBytes())
+                    .retain(connectionProperties.getLastWillRetain())
+                    .qos(MqttQos.fromCode(connectionProperties.getLastWillQOS()))
+                    .applyWillPublish();
+        }
+
+        final String username = connectionProperties.getUsername();
+        final String password = connectionProperties.getPassword();
+        if (username != null && password != null) {
+            connectBuilder.simpleAuth()
+                    .username(connectionProperties.getUsername())
+                    .password(password.getBytes(StandardCharsets.UTF_8))
+                    .applySimpleAuth();
+        }
+
+        final Mqtt5Connect mqtt5Connect = connectBuilder.build();
+        mqtt5BlockingClient.connect(mqtt5Connect);
+    }
+
+    @Override
+    public void disconnect(long disconnectTimeout) {
+        logger.debug("Disconnecting client");
+        // Currently it is not possible to set timeout for disconnect with HiveMQ Client. (Only connect timeout exists.)
+        mqtt5BlockingClient.disconnect();
+    }
+
+    @Override
+    public void close() {
+        // there is no paho's close equivalent in hivemq client
+    }
+
+    @Override
+    public void publish(String topic, StandardMqttMessage message) {
+        logger.debug("Publishing message to {} with QoS: {}", topic, message.getQos());
+
+        mqtt5BlockingClient.publishWith()
+                .topic(topic)
+                .payload(message.getPayload())
+                .retain(message.isRetained())
+                .qos(Objects.requireNonNull(MqttQos.fromCode(message.getQos())))
+                .send();
+    }
+
+    @Override
+    public void subscribe(String topicFilter, int qos) {
+        Objects.requireNonNull(callback, "callback should be set");
+
+        logger.debug("Subscribing to {} with QoS: {}", topicFilter, qos);
+
+        CompletableFuture<Mqtt5SubAck> ack = mqtt5BlockingClient.toAsync().subscribeWith()
+                .topicFilter(topicFilter)
+                .qos(Objects.requireNonNull(MqttQos.fromCode(qos)))
+                .callback(mqtt5Publish -> {
+                    final ReceivedMqttMessage receivedMessage = new ReceivedMqttMessage(
+                            mqtt5Publish.getPayloadAsBytes(),
+                            mqtt5Publish.getQos().getCode(),
+                            mqtt5Publish.isRetain(),
+                            mqtt5Publish.getTopic().toString());
+                    callback.messageArrived(receivedMessage);
+                })
+                .send();
+
+        // Setting "listener" callback is only possible with async client, though sending subscribe message
+        // should happen in a blocking way to make sure the processor is blocked until ack is not arrived.
+        ack.whenComplete((mqtt5SubAck, throwable) -> {
+            logger.debug("Received mqtt5 subscribe ack: {}", mqtt5SubAck.toString());
+
+            if (throwable != null) {
+                throw new MqttException("An error has occurred during sending subscribe message to broker", throwable);
+            }
+        });

Review Comment:
   `whenComplete()` registers an async callback, it does not block. The blocking way would be to use `ack.get()`.
   
   `whenComplete()` can also work but in this case:
   - in case of error (`throwable != null`), `logger.error()` must be used instead of throwing an exception in order to put the error message on the processor's bulletin
   - please fix the comment because this approach does not block



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952882215


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttConstants.java:
##########
@@ -67,14 +69,45 @@ public class MqttConstants {
      */
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_AUTO =
             new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_DEFAULT),
-                    "AUTO",
+                    "v3 AUTO",
                     "Start with v3.1.1 and fallback to v3.1.0 if not supported by a broker");
 
+    public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_500 =
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_5_0.getNumericValue()),
+                    "v5.0");
+
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_311 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1_1.getNumericValue()),
                     "v3.1.1");
 
     public static final AllowableValue ALLOWABLE_VALUE_MQTT_VERSION_310 =
-            new AllowableValue(String.valueOf(MqttConnectOptions.MQTT_VERSION_3_1),
+            new AllowableValue(String.valueOf(MqttVersion.MQTT_VERSION_3_1.getNumericValue()),
                     "v3.1.0");
+
+    public enum MqttVersion {
+        MQTT_VERSION_3_1(3),
+        MQTT_VERSION_3_1_1(4),
+        MQTT_VERSION_5_0(5);
+
+        private final int numericValue;
+
+        MqttVersion(int numericValue) {
+            this.numericValue = numericValue;
+        }
+
+        public int getNumericValue() {
+            return numericValue;
+        }
+    }
+
+    public enum SupportedSchemes {

Review Comment:
   Changed.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttClientFactory.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 com.hivemq.client.mqtt.mqtt5.Mqtt5Client;
+import com.hivemq.client.mqtt.mqtt5.Mqtt5ClientBuilder;
+import org.apache.nifi.processors.mqtt.adapters.HiveMqV5ClientAdapter;
+import org.apache.nifi.processors.mqtt.adapters.PahoMqttClientAdapter;
+import org.eclipse.paho.client.mqttv3.MqttClient;
+import org.eclipse.paho.client.mqttv3.MqttException;
+import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence;
+
+import static org.apache.nifi.processors.mqtt.common.MqttConstants.SupportedSchemes.SSL;
+import static org.apache.nifi.processors.mqtt.common.MqttConstants.SupportedSchemes.WS;
+import static org.apache.nifi.processors.mqtt.common.MqttConstants.SupportedSchemes.WSS;
+
+public class MqttClientFactory {
+    public NifiMqttClient create(MqttClientProperties clientProperties, MqttConnectionProperties connectionProperties) {
+        switch (clientProperties.getMqttVersion()) {
+            case 0:
+            case 3:
+            case 4:
+                return createPahoMqttV3ClientAdapter(clientProperties);
+            case 5:
+                return createHiveMqV5ClientAdapter(clientProperties, connectionProperties);

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] nandorsoma commented on a diff in pull request #6225: NIFI-10251 Add v5 protocol support for existing MQTT processors

Posted by GitBox <gi...@apache.org>.
nandorsoma commented on code in PR #6225:
URL: https://github.com/apache/nifi/pull/6225#discussion_r952884772


##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/NifiMqttMessage.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+public class NifiMqttMessage {

Review Comment:
   At start I wanted to do as minimal change as I can, that's why I left it in the original way. Nevertheless, you are right, I changed it.



##########
nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/common/MqttClientProperties.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 java.net.URI;
+
+public class MqttClientProperties {
+    private URI brokerURI;
+    private String clientID;

Review Comment:
   Changed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org