adam-markovics commented on code in PR #1432:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1432#discussion_r1041315835


##########
extensions/mqtt/processors/PublishMQTT.cpp:
##########
@@ -34,76 +34,229 @@ void PublishMQTT::initialize() {
   setSupportedRelationships(relationships());
 }
 
-void PublishMQTT::onSchedule(const std::shared_ptr<core::ProcessContext> 
&context, const std::shared_ptr<core::ProcessSessionFactory> &factory) {
+void PublishMQTT::readProperties(const std::shared_ptr<core::ProcessContext>& 
context) {
+  if (!context->getProperty(Topic).has_value()) {
+    logger_->log_error("PublishMQTT: could not get Topic");
+  }
+
   if (const auto retain_opt = context->getProperty<bool>(Retain)) {
     retain_ = *retain_opt;
   }
   logger_->log_debug("PublishMQTT: Retain [%d]", retain_);
 
-  AbstractMQTTProcessor::onSchedule(context, factory);
+  if (const auto message_expiry_interval = 
context->getProperty<core::TimePeriodValue>(MessageExpiryInterval)) {
+    message_expiry_interval_ = 
std::chrono::duration_cast<std::chrono::seconds>(message_expiry_interval->getMilliseconds());
+    logger_->log_debug("PublishMQTT: MessageExpiryInterval [%" PRId64 "] s", 
int64_t{message_expiry_interval_->count()});
+  }
+
+  in_flight_message_counter_.setMqttVersion(mqtt_version_);
+  in_flight_message_counter_.setQoS(qos_);
 }
 
-void PublishMQTT::onTrigger(const std::shared_ptr<core::ProcessContext>& 
/*context*/, const std::shared_ptr<core::ProcessSession> &session) {
-  // reconnect if needed
-  reconnect();
+void PublishMQTT::onTriggerImpl(const std::shared_ptr<core::ProcessContext>& 
context, const std::shared_ptr<core::ProcessSession> &session) {
+  std::shared_ptr<core::FlowFile> flow_file = session->get();
 
-  if (!MQTTAsync_isConnected(client_)) {
-    logger_->log_error("Could not publish to MQTT broker because disconnected 
to %s", uri_);
-    yield();
+  if (!flow_file) {
     return;
   }
 
-  std::shared_ptr<core::FlowFile> flowFile = session->get();
+  // broker's Receive Maximum can change after reconnect
+  
in_flight_message_counter_.setMax(broker_receive_maximum_.value_or(MQTT_MAX_RECEIVE_MAXIMUM));
 
-  if (!flowFile) {
-    return;
+  const auto topic = getTopic(context, flow_file);
+  try {
+    const auto result = session->readBuffer(flow_file);
+    if (result.status < 0 || !sendMessage(result.buffer, topic, 
getContentType(context, flow_file), flow_file)) {
+      logger_->log_error("Failed to send flow file [%s] to MQTT topic '%s' on 
broker %s", flow_file->getUUIDStr(), topic, uri_);
+      session->transfer(flow_file, Failure);
+      return;
+    }
+    logger_->log_debug("Sent flow file [%s] with length %" PRId64 " to MQTT 
topic '%s' on broker %s", flow_file->getUUIDStr(), result.status, topic, uri_);
+    session->transfer(flow_file, Success);
+  } catch (const Exception& ex) {
+    logger_->log_error("Failed to send flow file [%s] to MQTT topic '%s' on 
broker %s, exception string: '%s'", flow_file->getUUIDStr(), topic, uri_, 
ex.what());
+    session->transfer(flow_file, Failure);
+  }
+}
+
+bool PublishMQTT::sendMessage(const std::vector<std::byte>& buffer, const 
std::string& topic, const std::string& content_type, const 
std::shared_ptr<core::FlowFile>& flow_file) {
+  if (buffer.size() > 268'435'455) {
+    logger_->log_error("Sending message failed because MQTT limit maximum 
packet size [268'435'455] is exceeded by FlowFile of [%zu]", buffer.size());
   }
 
-  PublishMQTT::ReadCallback callback(this, flowFile->getSize(), max_seg_size_, 
topic_, client_, gsl::narrow<int>(qos_), retain_);
-  session->read(flowFile, std::ref(callback));
-  if (callback.status_ < 0) {
-    logger_->log_error("Failed to send flow to MQTT topic %s", topic_);
-    session->transfer(flowFile, Failure);
+  if (maximum_packet_size_.has_value() && buffer.size() > 
*(maximum_packet_size_)) {
+    logger_->log_error("Sending message failed because broker-requested 
maximum packet size [%" PRIu32 "] is exceeded by FlowFile of [%zu]",
+                                   *maximum_packet_size_, buffer.size());
+  }
+
+  MQTTAsync_message message_to_publish = MQTTAsync_message_initializer;
+  message_to_publish.payload = const_cast<std::byte*>(buffer.data());
+  message_to_publish.payloadlen = buffer.size();
+  message_to_publish.qos = qos_.value();
+  message_to_publish.retained = retain_;
+
+  setMqtt5Properties(message_to_publish, content_type, flow_file);
+
+  MQTTAsync_responseOptions response_options = 
MQTTAsync_responseOptions_initializer;
+  if (mqtt_version_ == MqttVersions::V_5_0) {
+    response_options.onSuccess5 = sendSuccess5;
+    response_options.onFailure5 = sendFailure5;
   } else {
-    logger_->log_debug("Sent flow with length %d to MQTT topic %s", 
callback.read_size_, topic_);
-    session->transfer(flowFile, Success);
-  }
-}
-
-int64_t PublishMQTT::ReadCallback::operator()(const 
std::shared_ptr<io::InputStream>& stream) {
-  if (flow_size_ < max_seg_size_)
-    max_seg_size_ = flow_size_;
-  gsl_Expects(max_seg_size_ < 
gsl::narrow<uint64_t>(std::numeric_limits<int>::max()));
-  std::vector<std::byte> buffer(max_seg_size_);
-  read_size_ = 0;
-  status_ = 0;
-  while (read_size_ < flow_size_) {
-    // MQTTClient_message::payloadlen is int, so we can't handle 2GB+
-    const auto readRet = stream->read(buffer);
-    if (io::isError(readRet)) {
-      status_ = -1;
-      return gsl::narrow<int64_t>(read_size_);
+    response_options.onSuccess = sendSuccess;
+    response_options.onFailure = sendFailure;
+  }
+
+  // save context for callback
+  std::packaged_task<bool(bool, std::optional<int>, 
std::optional<MQTTReasonCodes>)> send_finished_task(
+          [this] (const bool success, const std::optional<int> response_code, 
const std::optional<MQTTReasonCodes> reason_code) {
+            return notify(success, response_code, reason_code);
+          });
+  response_options.context = &send_finished_task;
+
+  in_flight_message_counter_.increase();
+
+  const int error_code = MQTTAsync_sendMessage(client_, topic.c_str(), 
&message_to_publish, &response_options);
+  if (error_code != MQTTASYNC_SUCCESS) {
+    logger_->log_error("MQTTAsync_sendMessage failed on topic '%s', MQTT 
broker %s with error code [%d]", topic, uri_, error_code);
+    // early fail, sending attempt did not succeed, no need to wait for 
callback
+    in_flight_message_counter_.decrease();
+    return false;
+  }
+
+  return send_finished_task.get_future().get();
+}
+
+void PublishMQTT::checkProperties() {
+  if ((mqtt_version_ == MqttVersions::V_3_1_0 || mqtt_version_ == 
MqttVersions::V_3_1_1 || mqtt_version_ == MqttVersions::V_3X_AUTO)) {
+    if (isPropertyExplicitlySet(MessageExpiryInterval)) {
+      logger_->log_warn("MQTT 3.x specification does not support Message 
Expiry Intervals. Property is not used.");
+    }
+    if (isPropertyExplicitlySet(ContentType)) {
+      logger_->log_warn("MQTT 3.x specification does not support Content 
Types. Property is not used.");
+    }
+  }
+}
+
+void PublishMQTT::checkBrokerLimitsImpl() {
+  if (retain_available_.has_value() && !*retain_available_ && retain_) {

Review Comment:
   Done.



-- 
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-unsubscr...@nifi.apache.org

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

Reply via email to