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


##########
sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIOProducer.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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 static 
org.apache.beam.sdk.io.jms.PublicationRetryPolicy.DEFAULT_PUBLICATION_RETRY_DURATION;
+
+import java.io.IOException;
+import javax.jms.Connection;
+import javax.jms.ConnectionFactory;
+import javax.jms.Destination;
+import javax.jms.JMSException;
+import javax.jms.Message;
+import javax.jms.MessageProducer;
+import javax.jms.Session;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.BackOff;
+import org.apache.beam.sdk.util.BackOffUtils;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.util.Sleeper;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({
+  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
+})
+public class JmsIOProducer<T> extends PTransform<PCollection<T>, 
WriteJmsResult<T>> {
+
+  public static final String CONNECTION_ERRORS_METRIC_NAME = 
"connectionErrors";
+  public static final String PUBLICATION_RETRIES_METRIC_NAME = 
"publicationRetries";
+  public static final String JMS_IO_PRODUCER_METRIC_NAME = 
JmsIOProducer.class.getCanonicalName();
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(JmsIOProducer.class);
+  private static final String PUBLISH_TO_JMS_STEP_NAME = "Publish to JMS";
+
+  private final JmsIO.Write<T> spec;
+  private final TupleTag<T> messagesTag;
+  private final TupleTag<T> failedMessagesTag;
+
+  JmsIOProducer(JmsIO.Write<T> spec) {
+    this.spec = spec;
+    this.messagesTag = new TupleTag<>();
+    this.failedMessagesTag = new TupleTag<>();
+  }
+
+  @Override
+  public WriteJmsResult<T> expand(PCollection<T> input) {
+    PCollectionTuple failedPublishedMessages =
+        input.apply(
+            PUBLISH_TO_JMS_STEP_NAME,
+            ParDo.of(new JmsIOProducerFn())
+                .withOutputTags(messagesTag, 
TupleTagList.of(failedMessagesTag)));
+    PCollection<T> failedMessages =
+        
failedPublishedMessages.get(failedMessagesTag).setCoder(input.getCoder());
+    failedPublishedMessages.get(messagesTag).setCoder(input.getCoder());
+    return WriteJmsResult.in(input.getPipeline(), failedMessagesTag, 
failedMessages);
+  }
+
+  private class JmsIOProducerFn extends DoFn<T, T> {
+
+    private transient @Initialized Session session;
+    private transient @Initialized Connection connection;
+    private transient @Initialized Destination destination;
+    private transient @Initialized MessageProducer producer;
+    private transient @Initialized FluentBackoff retryPublicationBackoff;
+
+    private final Counter connectionErrors =
+        Metrics.counter(JMS_IO_PRODUCER_METRIC_NAME, 
CONNECTION_ERRORS_METRIC_NAME);
+    private final Counter publicationRetries =
+        Metrics.counter(JMS_IO_PRODUCER_METRIC_NAME, 
PUBLICATION_RETRIES_METRIC_NAME);
+
+    @Setup
+    public void setup() {
+      retryPublicationBackoff =
+          FluentBackoff.DEFAULT
+              .withMaxRetries(0)

Review Comment:
   @Amraneze thanks for explanation. I get where the issue is. What I mean 
previously ""by default" was that, the sink should retry by default. Here I see 
if retrypolicy is not setup, there is still no retry (current behavior)
   
   I am suggesting follow the same pattern used elsewhere (e.g. 
https://github.com/apache/beam/blob/aba175259944caa75a7673b9e0ad430d8e5c68fe/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java#L2459-L2465)
   
   Specifically, name the retry policy as RetryConfiguration that is an 
internal class of JmsIO, and set a default retry configuration if user does not 
provide it. I see there is a substantial refactoring of the code in this PR 
which may not be necessary (producer can still be in JmsIO class) so users have 
a consistent experience working on different IO connectors of Beam. Finally 
thanks that the unit test has been implemented.



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