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


##########
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:
   I see where the data loss reported in #24971 came from. There should be a 
`@FinishBundle` annotated DoFn method that flushes producer and make sure it 
flushes successfully. Currently the next bundle could try to publish messages 
of the previous bundle remained in the producer's buffer. When connection fails 
it then retries on wrong data.
   
   And in Teardown close the producer, session, etc. If I remember correctly, 
the Exception in Teardown would not trigger retry. So the producer should 
already be flushed (in the last finish bundle) before teardown



-- 
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