Amraneze commented on code in PR #24973:
URL: https://github.com/apache/beam/pull/24973#discussion_r1087213526


##########
sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java:
##########
@@ -932,36 +1067,78 @@ public void setup() throws Exception {
           } else if (spec.getTopic() != null) {
             this.destination = session.createTopic(spec.getTopic());
           }
-
-          this.producer = this.session.createProducer(null);
+          this.producer = this.session.createProducer(this.destination);
         }
       }
 
       @ProcessElement
-      public void processElement(ProcessContext ctx) {
-        Destination destinationToSendTo = destination;
+      public void processElement(@Element T input, ProcessContext context) {
         try {
-          Message message = spec.getValueMapper().apply(ctx.element(), 
session);
-          if (spec.getTopicNameMapper() != null) {
-            destinationToSendTo =
-                
session.createTopic(spec.getTopicNameMapper().apply(ctx.element()));
+          publishMessage(input, context);
+        } catch (IOException | InterruptedException exception) {
+          LOG.error("Error while publishing the message", exception);
+          context.output(failedMessagesTag, input);
+          Thread.currentThread().interrupt();
+        }
+      }
+
+      private void publishMessage(T input, ProcessContext context)
+          throws IOException, InterruptedException {
+        Sleeper sleeper = Sleeper.DEFAULT;
+        Destination destinationToSendTo = destination;
+        BackOff backoff = checkStateNotNull(retryBackOff).backoff();
+        int publicationAttempt = 0;
+        while (publicationAttempt >= 0) {
+          publicationAttempt++;
+          try {
+            Message message = spec.getValueMapper().apply(input, session);
+            if (spec.getTopicNameMapper() != null) {
+              destinationToSendTo = 
session.createTopic(spec.getTopicNameMapper().apply(input));
+            }
+            producer.send(destinationToSendTo, message);
+            publicationAttempt = -1;
+          } catch (Exception exception) {
+            if (!BackOffUtils.next(sleeper, backoff)) {
+              LOG.error("The message wasn't published to topic {}", 
destinationToSendTo, exception);
+              context.output(failedMessagesTag, input);
+              publicationAttempt = -1;
+            } else {
+              publicationRetries.inc();
+              LOG.warn(
+                  "Error sending message on topic {}, retry attempt {}",
+                  destinationToSendTo,
+                  publicationAttempt,
+                  exception);
+            }
           }
-          producer.send(destinationToSendTo, message);
-        } catch (Exception ex) {
-          LOG.error("Error sending message on topic {}", destinationToSendTo);
-          ctx.output(failedMessageTag, ctx.element());
         }
       }
 
+      private void restartJmsConnection() throws JMSException {
+        teardown();
+        start();
+      }
+
       @Teardown
-      public void teardown() throws Exception {
-        producer.close();
-        producer = null;
-        session.close();
-        session = null;
-        connection.stop();
-        connection.close();
-        connection = null;
+      public void teardown() throws JMSException {
+        if (producer != null) {
+          producer.close();

Review Comment:
   With the current implementation of the producer, we don't have a way to 
flush the messages. Should we create an accumulative list of messages and when 
tearing down the DoFn we can flush those messages, what do you think ?
   
   >Note: When the connection is closed, the producer is no longer available, 
so we will need to recreate a connection, a session and the producer. Which 
means that during Finish bundle DoFn's lifecycle, we will need to recreate them 
to flush the messages. That's why I called the two functions `teardown` & 
`start` to make sure the producer is recreated in case of an exception even 
though it changes the mutation of the objects.
   
   <img width="466" alt="image" 
src="https://user-images.githubusercontent.com/28459763/214697936-91f1461a-9efc-44d1-8347-fcf62ac46f91.png";>



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to