Abacn commented on code in PR #24973:
URL: https://github.com/apache/beam/pull/24973#discussion_r1087198972
##########
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:
Yes, should re-throw the error then let the runner retry the bundle,
otherwise the bundle will still success but data not got written
--
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]