You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/03/31 21:17:19 UTC

[GitHub] [beam] lukecwik commented on a change in pull request #17163: [BEAM-13608] JmsIO dynamic topics feature

lukecwik commented on a change in pull request #17163:
URL: https://github.com/apache/beam/pull/17163#discussion_r840003385



##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function

Review comment:
       ```suggestion
        * {@link JmsIO.Write#withTopic(String)}. The user has to specify a {@link SerializableFunction}
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content

Review comment:
       ```suggestion
        * that takes {@code EventT} object as a parameter, and returns the topic name depending of the content
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content
+     * of the event object.
+     *
+     * <p>For instance:
+     * <pre>{@code
+     * SerializableFunction<SomeEventObject, String> topicNameMapper =
+     *         (e ->
+     *             return the topic name ;
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withTopicNameMapper(topicNameNapper)
+     *
+     * }</pre>
+     *
+     * @param topicNameMapper The function returning the dynamic topic name.
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withTopicNameMapper(SerializableFunction<EventT, String> topicNameMapper) {
+      checkArgument(topicNameMapper != null, "topicNameMapper can not be null");
+      return builder().setTopicNameMapper(topicNameMapper).build();
+    }
+
+    /**
+     * Map the Event object with a {@link javax.jms.Message}.

Review comment:
       ```suggestion
        * Map the {@code EventT} object to a {@link javax.jms.Message}.
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.

Review comment:
       Please update the above comment to reference that one of topic/queue/topicNameMapper must be set.

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content
+     * of the event object.
+     *
+     * <p>For instance:
+     * <pre>{@code
+     * SerializableFunction<SomeEventObject, String> topicNameMapper =
+     *         (e ->
+     *             return the topic name ;

Review comment:
       This code fragment seems broken.

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content
+     * of the event object.
+     *
+     * <p>For instance:
+     * <pre>{@code
+     * SerializableFunction<SomeEventObject, String> topicNameMapper =
+     *         (e ->
+     *             return the topic name ;
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withTopicNameMapper(topicNameNapper)
+     *
+     * }</pre>
+     *
+     * @param topicNameMapper The function returning the dynamic topic name.
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withTopicNameMapper(SerializableFunction<EventT, String> topicNameMapper) {
+      checkArgument(topicNameMapper != null, "topicNameMapper can not be null");
+      return builder().setTopicNameMapper(topicNameMapper).build();
+    }
+
+    /**
+     * Map the Event object with a {@link javax.jms.Message}.
+     *
+     * <p>For instance:
+     *
+     * <pre>{@code
+     * SerializableMapper<VehicleEvent> valueMapper = (e, s) -> {
+     *
+     *       try {
+     *         TextMessage msg = s.createTextMessage();
+     *         msg.setText(Mapper.MAPPER.toJson(e));
+     *         return msg;
+     *       } catch (JMSException ex) {
+     *         throw new JmsIOException("Error!!", ex);
+     *       }
+     *     };
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withValueMapper(valueNapper)
+     *
+     * }</pre>
+     *
+     * @param valueMapper The function returning the {@link javax.jms.Message}
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withValueMapper(SerializableMessageMapper<EventT> valueMapper) {
+      checkArgument(valueMapper != null, "valueMapper can not be null");
+      return builder().setValueMapper(valueMapper).build();
+    }
+
+    public Write<EventT> withCoder(Coder<EventT> coder) {
+      checkArgument(coder != null, "coder can not be null");
+      return builder().setCoder(coder).build();
+    }
+
     @Override
-    public PDone expand(PCollection<String> input) {
+    public WriteJmsResult<EventT> expand(PCollection<EventT> input) {
       checkArgument(getConnectionFactory() != null, "withConnectionFactory() is required");
+      checkArgument(getCoder() != null, "withCoder() is required");
       checkArgument(
-          getQueue() != null || getTopic() != null,
-          "Either withQueue(queue) or withTopic(topic) is required");
+          getTopicNameMapper() != null || getQueue() != null || getTopic() != null,
+          "Either withTopicNameMapper(topicNameMapper) or withQueue(queue) or withTopic(topic) is required");
+      boolean exclusiveTopicQueue = isExclusiveTopicQueue();
       checkArgument(
-          getQueue() == null || getTopic() == null,
-          "withQueue(queue) and withTopic(topic) are exclusive");
+          exclusiveTopicQueue,
+          "withQueue(queue) and withTopic(topic) and withTopicNameMapper(function) are exclusive");
+      checkArgument(getValueMapper() != null, "withValueeMapper() is required");

Review comment:
       ```suggestion
         checkArgument(getValueMapper() != null, "withValueMapper() is required");
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIOException.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public class JmsIOException extends RuntimeException {

Review comment:
       Why did you want to implement this class, existing tests and documentation seem like they would work fine with reusing Java's RuntimeException as is.

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/WriteJmsResult.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.util.Map;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/**
+ * Return type of {@link JmsIO.Write} transform. All messages in error are identified by: -
+ * TupleTag<EventT> failedMessageTag - PCollection<EventT> failedMessages

Review comment:
       nit: comment formatting

##########
File path: sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
##########
@@ -255,6 +259,93 @@ public void testWriteMessage() throws Exception {
     assertEquals(100, count);
   }
 
+  @Test
+  public void testWriteMessageWithError() throws Exception {
+
+    ArrayList<String> data = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      data.add("Message " + i);
+    }
+    String messageOne = "Message 1";
+    String messageTwo = "Message 2";
+
+    WriteJmsResult<String> output =
+        pipeline
+            .apply(Create.of(data))
+            .apply(
+                JmsIO.<String>write()
+                    .withConnectionFactory(connectionFactory)
+                    .withValueMapper(new TextMessageMapperWithError())
+                    .withQueue(QUEUE)
+                    .withCoder(SerializableCoder.of(String.class))
+                    .withUsername(USERNAME)
+                    .withPassword(PASSWORD));
+
+    PAssert.that(output.getFailedMessages()).containsInAnyOrder(messageOne, messageTwo);
+
+    pipeline.run();
+
+    Connection connection = connectionFactory.createConnection(USERNAME, PASSWORD);
+    connection.start();
+    Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+    MessageConsumer consumer = session.createConsumer(session.createQueue(QUEUE));
+    int count = 0;
+    while (consumer.receive(1000) != null) {
+      count++;
+    }
+    assertEquals(98, count);
+    assertTrue(output.getPipeline().equals(pipeline));
+  }
+
+  @Test
+  public void testWriteDynamicMessage() throws Exception {
+
+    Connection connection = connectionFactory.createConnection(USERNAME, PASSWORD);
+    connection.start();
+    Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+    MessageConsumer consumerOne = session.createConsumer(session.createTopic("Topic_One"));
+    MessageConsumer consumerTwo = session.createConsumer(session.createTopic("Topic_Two"));
+    ArrayList<TestEvent> data = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      data.add(new TestEvent("Topic_One", "Message One " + i));
+    }
+    for (int i = 0; i < 100; i++) {
+      data.add(new TestEvent("Topic_Two", "Message Two " + i));
+    }
+    pipeline
+        .apply(Create.of(data))
+        .apply(
+            JmsIO.<TestEvent>write()
+                .withConnectionFactory(connectionFactory)
+                .withUsername(USERNAME)
+                .withPassword(PASSWORD)
+                .withCoder(SerializableCoder.of(TestEvent.class))
+                .withTopicNameMapper(e -> e.getTopicName())
+                .withValueMapper(
+                    (e, s) -> {
+                      try {
+                        TextMessage msg = s.createTextMessage();
+                        msg.setText(e.getValue());
+                        return msg;
+                      } catch (JMSException ex) {
+                        throw new JmsIOException("Error writing TextMessage", ex);
+                      }
+                    }));
+
+    pipeline.run();
+
+    int count = 0;
+    while (consumerOne.receive(1000) != null) {
+      count++;
+    }
+    assertEquals(100, count);
+
+    while (consumerTwo.receive(1000) != null) {
+      count++;
+    }
+    assertEquals(200, count);

Review comment:
       ```suggestion
       count = 0;
       while (consumerTwo.receive(1000) != null) {
         count++;
       }
       assertEquals(100, count);
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content
+     * of the event object.
+     *
+     * <p>For instance:
+     * <pre>{@code
+     * SerializableFunction<SomeEventObject, String> topicNameMapper =
+     *         (e ->
+     *             return the topic name ;
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withTopicNameMapper(topicNameNapper)
+     *
+     * }</pre>
+     *
+     * @param topicNameMapper The function returning the dynamic topic name.
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withTopicNameMapper(SerializableFunction<EventT, String> topicNameMapper) {
+      checkArgument(topicNameMapper != null, "topicNameMapper can not be null");
+      return builder().setTopicNameMapper(topicNameMapper).build();
+    }
+
+    /**
+     * Map the Event object with a {@link javax.jms.Message}.
+     *
+     * <p>For instance:
+     *
+     * <pre>{@code
+     * SerializableMapper<VehicleEvent> valueMapper = (e, s) -> {
+     *
+     *       try {
+     *         TextMessage msg = s.createTextMessage();
+     *         msg.setText(Mapper.MAPPER.toJson(e));
+     *         return msg;
+     *       } catch (JMSException ex) {
+     *         throw new JmsIOException("Error!!", ex);
+     *       }
+     *     };
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withValueMapper(valueNapper)
+     *
+     * }</pre>
+     *
+     * @param valueMapper The function returning the {@link javax.jms.Message}
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withValueMapper(SerializableMessageMapper<EventT> valueMapper) {
+      checkArgument(valueMapper != null, "valueMapper can not be null");
+      return builder().setValueMapper(valueMapper).build();
+    }
+
+    public Write<EventT> withCoder(Coder<EventT> coder) {
+      checkArgument(coder != null, "coder can not be null");
+      return builder().setCoder(coder).build();
+    }
+
     @Override
-    public PDone expand(PCollection<String> input) {
+    public WriteJmsResult<EventT> expand(PCollection<EventT> input) {
       checkArgument(getConnectionFactory() != null, "withConnectionFactory() is required");
+      checkArgument(getCoder() != null, "withCoder() is required");

Review comment:
       Why not use input.getCoder()?
   
   This would allow you to get rid of withCoder().

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -668,7 +693,7 @@ public Write withConnectionFactory(ConnectionFactory connectionFactory) {
      * @param queue The JMS queue name where to send messages to.

Review comment:
       Please update the above comment to reference that one of topic/queue/topicNameMapper must be set.

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content
+     * of the event object.
+     *
+     * <p>For instance:
+     * <pre>{@code
+     * SerializableFunction<SomeEventObject, String> topicNameMapper =
+     *         (e ->
+     *             return the topic name ;
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withTopicNameMapper(topicNameNapper)
+     *
+     * }</pre>
+     *
+     * @param topicNameMapper The function returning the dynamic topic name.
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withTopicNameMapper(SerializableFunction<EventT, String> topicNameMapper) {
+      checkArgument(topicNameMapper != null, "topicNameMapper can not be null");
+      return builder().setTopicNameMapper(topicNameMapper).build();
+    }
+
+    /**
+     * Map the Event object with a {@link javax.jms.Message}.
+     *
+     * <p>For instance:
+     *
+     * <pre>{@code
+     * SerializableMapper<VehicleEvent> valueMapper = (e, s) -> {
+     *
+     *       try {
+     *         TextMessage msg = s.createTextMessage();
+     *         msg.setText(Mapper.MAPPER.toJson(e));
+     *         return msg;
+     *       } catch (JMSException ex) {
+     *         throw new JmsIOException("Error!!", ex);
+     *       }
+     *     };
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withValueMapper(valueNapper)
+     *
+     * }</pre>
+     *
+     * @param valueMapper The function returning the {@link javax.jms.Message}
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withValueMapper(SerializableMessageMapper<EventT> valueMapper) {
+      checkArgument(valueMapper != null, "valueMapper can not be null");
+      return builder().setValueMapper(valueMapper).build();
+    }
+
+    public Write<EventT> withCoder(Coder<EventT> coder) {
+      checkArgument(coder != null, "coder can not be null");
+      return builder().setCoder(coder).build();
+    }
+
     @Override
-    public PDone expand(PCollection<String> input) {
+    public WriteJmsResult<EventT> expand(PCollection<EventT> input) {
       checkArgument(getConnectionFactory() != null, "withConnectionFactory() is required");
+      checkArgument(getCoder() != null, "withCoder() is required");
       checkArgument(
-          getQueue() != null || getTopic() != null,
-          "Either withQueue(queue) or withTopic(topic) is required");
+          getTopicNameMapper() != null || getQueue() != null || getTopic() != null,
+          "Either withTopicNameMapper(topicNameMapper) or withQueue(queue) or withTopic(topic) is required");
+      boolean exclusiveTopicQueue = isExclusiveTopicQueue();
       checkArgument(
-          getQueue() == null || getTopic() == null,
-          "withQueue(queue) and withTopic(topic) are exclusive");
+          exclusiveTopicQueue,
+          "withQueue(queue) and withTopic(topic) and withTopicNameMapper(function) are exclusive");

Review comment:
       ```suggestion
             "Only one of withQueue(queue), withTopic(topic), or withTopicNameMapper(function) must be set.");
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/TextMessageMapper.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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 javax.jms.JMSException;
+import javax.jms.Message;
+import javax.jms.Session;
+import javax.jms.TextMessage;
+
+/**
+ * Default TextMessageMapper Takes a {@link String} value, a {@link javax.jms.Session} and returns a

Review comment:
       ```suggestion
    * The TextMessageMapper takes a {@link String} value, a {@link javax.jms.Session} and returns a
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -746,21 +860,37 @@ public void setup() throws Exception {
           this.connection.start();
           // false means we don't use JMS transaction.
           this.session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
-          Destination destination;
+
           if (spec.getQueue() != null) {
-            destination = session.createQueue(spec.getQueue());
-          } else {
-            destination = session.createTopic(spec.getTopic());
+            this.destination = session.createQueue(spec.getQueue());
+          } else if (spec.getTopic() != null) {
+            this.destination = session.createTopic(spec.getTopic());
           }
-          this.producer = this.session.createProducer(destination);
+
+          this.producer = this.session.createProducer(null);
         }
       }
 
       @ProcessElement
-      public void processElement(ProcessContext ctx) throws Exception {
-        String value = ctx.element();
-        TextMessage message = session.createTextMessage(value);
-        producer.send(message);
+      public void processElement(ProcessContext ctx) {
+        Destination dynamicDestination = null;
+        try {
+          Message message = spec.getValueMapper().apply(ctx.element(), session);
+          if (spec.getTopicNameMapper() != null) {
+            dynamicDestination =
+                session.createTopic(spec.getTopicNameMapper().apply(ctx.element()));
+            producer.send(dynamicDestination, message);
+
+          } else {
+            producer.send(destination, message);
+          }
+
+        } catch (Exception ex) {
+          LOG.error(
+              "Error sending message on topic {}",
+              dynamicDestination != null ? dynamicDestination : destination);

Review comment:
       ```suggestion
             LOG.error(
                 "Error sending message on topic {}",
                 destinationToSendTo);
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -616,21 +628,34 @@ public void close() throws IOException {
 
     abstract @Nullable String getPassword();
 
-    abstract Builder builder();
+    abstract @Nullable SerializableMessageMapper<EventT> getValueMapper();
+
+    abstract @Nullable SerializableFunction<EventT, String> getTopicNameMapper();
+
+    abstract @Nullable Coder<EventT> getCoder();
+
+    abstract Builder<EventT> builder();
 
     @AutoValue.Builder
-    abstract static class Builder {
-      abstract Builder setConnectionFactory(ConnectionFactory connectionFactory);
+    abstract static class Builder<EventT> {
+      abstract Builder<EventT> setConnectionFactory(ConnectionFactory connectionFactory);
+
+      abstract Builder<EventT> setQueue(String queue);
+
+      abstract Builder<EventT> setTopic(String topic);
+
+      abstract Builder<EventT> setUsername(String username);
 
-      abstract Builder setQueue(String queue);
+      abstract Builder<EventT> setPassword(String password);
 
-      abstract Builder setTopic(String topic);
+      abstract Builder<EventT> setValueMapper(SerializableMessageMapper<EventT> valueMapper);

Review comment:
       Instead of having a topic parameter, would it make sense to have setTopic invoke setTopicNameMapper with a SerializableFunction that returns topic?
   
   Would reduce the validation complexity since we only have to check whether queue has been set or topicNameMapper

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/SerializableMessageMapper.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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 javax.jms.Message;
+import javax.jms.Session;
+import org.apache.beam.sdk.transforms.SerializableBiFunction;
+
+/**
+ * SerializableMessageMapper Takes a {@link String} value, a {@link javax.jms.Session} and returns a
+ * {@link javax.jms.Message}.
+ */
+public interface SerializableMessageMapper<EventT>

Review comment:
       I think it would be ok to use SerializableBiFunction directly within JmsIO instead of defining this interface.

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content
+     * of the event object.
+     *
+     * <p>For instance:
+     * <pre>{@code
+     * SerializableFunction<SomeEventObject, String> topicNameMapper =
+     *         (e ->
+     *             return the topic name ;
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withTopicNameMapper(topicNameNapper)
+     *
+     * }</pre>
+     *
+     * @param topicNameMapper The function returning the dynamic topic name.
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withTopicNameMapper(SerializableFunction<EventT, String> topicNameMapper) {
+      checkArgument(topicNameMapper != null, "topicNameMapper can not be null");
+      return builder().setTopicNameMapper(topicNameMapper).build();
+    }
+
+    /**
+     * Map the Event object with a {@link javax.jms.Message}.
+     *
+     * <p>For instance:
+     *
+     * <pre>{@code
+     * SerializableMapper<VehicleEvent> valueMapper = (e, s) -> {
+     *
+     *       try {
+     *         TextMessage msg = s.createTextMessage();
+     *         msg.setText(Mapper.MAPPER.toJson(e));
+     *         return msg;
+     *       } catch (JMSException ex) {
+     *         throw new JmsIOException("Error!!", ex);
+     *       }
+     *     };
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withValueMapper(valueNapper)
+     *
+     * }</pre>
+     *
+     * @param valueMapper The function returning the {@link javax.jms.Message}
+     * @return The corresponding {@link JmsIO.Write}.
+     */
+    public Write<EventT> withValueMapper(SerializableMessageMapper<EventT> valueMapper) {
+      checkArgument(valueMapper != null, "valueMapper can not be null");
+      return builder().setValueMapper(valueMapper).build();
+    }
+
+    public Write<EventT> withCoder(Coder<EventT> coder) {
+      checkArgument(coder != null, "coder can not be null");
+      return builder().setCoder(coder).build();
+    }
+
     @Override
-    public PDone expand(PCollection<String> input) {
+    public WriteJmsResult<EventT> expand(PCollection<EventT> input) {
       checkArgument(getConnectionFactory() != null, "withConnectionFactory() is required");
+      checkArgument(getCoder() != null, "withCoder() is required");
       checkArgument(
-          getQueue() != null || getTopic() != null,
-          "Either withQueue(queue) or withTopic(topic) is required");
+          getTopicNameMapper() != null || getQueue() != null || getTopic() != null,
+          "Either withTopicNameMapper(topicNameMapper) or withQueue(queue) or withTopic(topic) is required");

Review comment:
       ```suggestion
             "Either withTopicNameMapper(topicNameMapper), withQueue(queue), or withTopic(topic) is required");
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -690,47 +715,136 @@ public Write withQueue(String queue) {
      * @param topic The JMS topic name.
      * @return The corresponding {@link JmsIO.Read}.
      */
-    public Write withTopic(String topic) {
+    public Write<EventT> withTopic(String topic) {
       checkArgument(topic != null, "topic can not be null");
       return builder().setTopic(topic).build();
     }
 
     /** Define the username to connect to the JMS broker (authenticated). */
-    public Write withUsername(String username) {
+    public Write<EventT> withUsername(String username) {
       checkArgument(username != null, "username can not be null");
       return builder().setUsername(username).build();
     }
 
     /** Define the password to connect to the JMS broker (authenticated). */
-    public Write withPassword(String password) {
+    public Write<EventT> withPassword(String password) {
       checkArgument(password != null, "password can not be null");
       return builder().setPassword(password).build();
     }
 
+    /**
+     * Specify the JMS topic destination name where to send messages to dynamically. The {@link
+     * JmsIO.Write} acts as a publisher on the topic.
+     *
+     * <p>This method is exclusive with {@link JmsIO.Write#withQueue(String) and
+     *{@link JmsIO.Write#withTopic(String)}. The user has to specify a Serializable function
+     * that takes the Event Object as parameter, and returns the topîc name depending of the content
+     * of the event object.
+     *
+     * <p>For instance:
+     * <pre>{@code
+     * SerializableFunction<SomeEventObject, String> topicNameMapper =
+     *         (e ->
+     *             return the topic name ;
+     *
+     * }</pre>
+     *
+     * <pre>{@code
+     * .apply(JmsIO.write().withTopicNameMapper(topicNameNapper)
+     *

Review comment:
       ```suggestion
   ```

##########
File path: sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
##########
@@ -255,6 +259,93 @@ public void testWriteMessage() throws Exception {
     assertEquals(100, count);
   }
 
+  @Test
+  public void testWriteMessageWithError() throws Exception {
+
+    ArrayList<String> data = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      data.add("Message " + i);
+    }
+    String messageOne = "Message 1";
+    String messageTwo = "Message 2";
+
+    WriteJmsResult<String> output =
+        pipeline
+            .apply(Create.of(data))
+            .apply(
+                JmsIO.<String>write()
+                    .withConnectionFactory(connectionFactory)
+                    .withValueMapper(new TextMessageMapperWithError())
+                    .withQueue(QUEUE)
+                    .withCoder(SerializableCoder.of(String.class))
+                    .withUsername(USERNAME)
+                    .withPassword(PASSWORD));
+
+    PAssert.that(output.getFailedMessages()).containsInAnyOrder(messageOne, messageTwo);
+
+    pipeline.run();
+
+    Connection connection = connectionFactory.createConnection(USERNAME, PASSWORD);
+    connection.start();
+    Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+    MessageConsumer consumer = session.createConsumer(session.createQueue(QUEUE));
+    int count = 0;
+    while (consumer.receive(1000) != null) {
+      count++;
+    }
+    assertEquals(98, count);
+    assertTrue(output.getPipeline().equals(pipeline));
+  }
+
+  @Test
+  public void testWriteDynamicMessage() throws Exception {
+

Review comment:
       ```suggestion
   ```

##########
File path: sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
##########
@@ -255,6 +259,93 @@ public void testWriteMessage() throws Exception {
     assertEquals(100, count);
   }
 
+  @Test
+  public void testWriteMessageWithError() throws Exception {
+

Review comment:
       ```suggestion
   ```

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -48,13 +49,18 @@
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;

Review comment:
       We'll want to update https://github.com/apache/beam/blob/master/CHANGES.md to state that JmsIO is changing such that a value mapper is required. For users to get the existing behavior, they should invoke JmsIO.write().withValueMapper(new TextMessageMapper())
   
   We should also highlight that writing to JmsIO is gaining the ability to map to multiple JMS message types and also that will support dynamic destinations.

##########
File path: sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
##########
@@ -746,21 +860,37 @@ public void setup() throws Exception {
           this.connection.start();
           // false means we don't use JMS transaction.
           this.session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
-          Destination destination;
+
           if (spec.getQueue() != null) {
-            destination = session.createQueue(spec.getQueue());
-          } else {
-            destination = session.createTopic(spec.getTopic());
+            this.destination = session.createQueue(spec.getQueue());
+          } else if (spec.getTopic() != null) {
+            this.destination = session.createTopic(spec.getTopic());
           }
-          this.producer = this.session.createProducer(destination);
+
+          this.producer = this.session.createProducer(null);
         }
       }
 
       @ProcessElement
-      public void processElement(ProcessContext ctx) throws Exception {
-        String value = ctx.element();
-        TextMessage message = session.createTextMessage(value);
-        producer.send(message);
+      public void processElement(ProcessContext ctx) {
+        Destination dynamicDestination = null;
+        try {
+          Message message = spec.getValueMapper().apply(ctx.element(), session);
+          if (spec.getTopicNameMapper() != null) {
+            dynamicDestination =
+                session.createTopic(spec.getTopicNameMapper().apply(ctx.element()));
+            producer.send(dynamicDestination, message);
+
+          } else {
+            producer.send(destination, message);
+          }
+

Review comment:
       ```suggestion
           Destination destinationToSendTo = destination;
           try {
             Message message = spec.getValueMapper().apply(ctx.element(), session);
             if (spec.getTopicNameMapper() != null) {
               destinationToSendTo =
                   session.createTopic(spec.getTopicNameMapper().apply(ctx.element()));
             }
             producer.send(destinationToSendTo, message);
   ```

##########
File path: sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
##########
@@ -555,4 +646,39 @@ private ConnectionFactory withSlowAcks(ConnectionFactory factory, long delay) {
               return result;
             });
   }
+
+  private static class TestEvent implements Serializable {
+    private final String topicName;
+    private final String value;
+
+    private TestEvent(String topicName, String value) {
+      this.topicName = topicName;
+      this.value = value;
+    }
+
+    private String getTopicName() {
+      return this.topicName;
+    }
+
+    private String getValue() {
+      return this.value;
+    }
+  }
+
+  private static class TextMessageMapperWithError implements SerializableMessageMapper<String> {
+

Review comment:
       ```suggestion
   ```




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org