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


##########
CHANGES.md:
##########
@@ -60,6 +60,7 @@
 
 * Support for X source added (Java/Python) 
([#X](https://github.com/apache/beam/issues/X)).
 * MongoDB IO connector added (Go) 
([#24575](https://github.com/apache/beam/issues/24575)).
+* Added in JmsIO a retry policy for failed publications (Java) 
([#24971](https://github.com/apache/beam/issues/24971)).

Review Comment:
   v2.45.0 has already been cut last week. If need to get into 2.45.0 please 
contact @johnjcasey; otherwise this update should fall in the section of v2.46.0



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

Review Comment:
   DoFn override methods (`@StartBundle`, `@TearDown` etc) should not be called 
elsewhere. This messes up the DoFn life cycle. Also, `@Teardown` corresponds to 
`@Setup` while `@StartBundle` corresponds to `@FinishBundle`
   
   If needed setup and closing connections can be declared in another private 
synchronized method, and can be called both in DoFn override methods and other 
helper methods.



##########
sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/FakeConnection.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.beam.sdk.io.jms;
+
+import java.io.Serializable;
+import javax.jms.Connection;
+import javax.jms.ConnectionConsumer;
+import javax.jms.ConnectionMetaData;
+import javax.jms.Destination;
+import javax.jms.ExceptionListener;
+import javax.jms.ServerSessionPool;
+import javax.jms.Session;
+import javax.jms.Topic;
+import org.apache.activemq.AlreadyClosedException;
+import org.apache.activemq.ConnectionFailedException;
+import org.mockito.Mockito;
+
+public class FakeConnection implements Connection, Serializable {

Review Comment:
   (minor) though been test class, nice to have a javadoc saying what this mock 
class achieves.



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



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