[
https://issues.apache.org/jira/browse/BEAM-1240?focusedWorklogId=158571&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-158571
]
ASF GitHub Bot logged work on BEAM-1240:
----------------------------------------
Author: ASF GitHub Bot
Created on: 25/Oct/18 07:56
Start Date: 25/Oct/18 07:56
Worklog Time Spent: 10m
Work Description: echauchot closed pull request #1729: [BEAM-1240] Create
RabbitMqIO
URL: https://github.com/apache/beam/pull/1729
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git a/sdks/java/io/rabbitmq/build.gradle
b/sdks/java/io/rabbitmq/build.gradle
new file mode 100644
index 00000000000..bef18f9309e
--- /dev/null
+++ b/sdks/java/io/rabbitmq/build.gradle
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+apply plugin: org.apache.beam.gradle.BeamModulePlugin
+applyJavaNature()
+
+description = "Apache Beam :: SDKs :: Java :: IO :: RabbitMQ"
+ext.summary = "IO to read and write to a RabbitMQ broker."
+
+dependencies {
+ compile library.java.guava
+ shadow project(path: ":beam-sdks-java-core", configuration: "shadow")
+ shadow library.java.joda_time
+ shadow "com.rabbitmq:amqp-client:4.6.0"
+ testCompile project(path: ":beam-runners-direct-java", configuration:
"shadow")
+ testCompile "org.apache.qpid:qpid-broker:0.28"
+ testCompile "org.apache.qpid:qpid-broker-core:0.28"
+ testCompile library.java.junit
+ testCompile library.java.hamcrest_core
+ testCompile library.java.hamcrest_library
+ testCompile library.java.slf4j_api
+ testCompile library.java.slf4j_jdk14
+}
diff --git
a/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
new file mode 100644
index 00000000000..d7058120388
--- /dev/null
+++
b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
@@ -0,0 +1,641 @@
+/*
+ * 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.rabbitmq;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.MessageProperties;
+import com.rabbitmq.client.QueueingConsumer;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.TimeoutException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A IO to publish or consume messages with a RabbitMQ broker.
+ *
+ * <h3>Consuming messages from RabbitMQ server</h3>
+ *
+ * <p>{@link RabbitMqIO} {@link Read} returns an unbounded {@link PCollection}
containing RabbitMQ
+ * messages body (as {@code byte[]}) wrapped as {@link RabbitMqMessage}.
+ *
+ * <p>To configure a RabbitMQ source, you have to provide a RabbitMQ {@code
URI} to connect to a
+ * RabbitMQ broker. The following example illustrates various options for
configuring the source,
+ * reading from the queue:
+ *
+ * <pre>{@code
+ * PCollection<RabbitMqMessage> messages = pipeline.apply(
+ *
RabbitMqIO.read().withUri("amqp://user:password@localhost:5672").withQueue("QUEUE"))
+ *
+ * }</pre>
+ *
+ * <p>It's also possible to read from an exchange (providing the exchange type
and routing key)
+ * instead of directly from a queue:
+ *
+ * <pre>{@code
+ * PCollection<RabbitMqMessage> messages = pipeline.apply(
+ *
RabbitMqIO.read().withUri("amqp://user:password@localhost:5672").withExchange("EXCHANGE",
"fanout", "QUEUE"));
+ * }</pre>
+ *
+ * <h3>Publishing messages to RabbitMQ server</h3>
+ *
+ * <p>{@link RabbitMqIO} {@link Write} can send {@link RabbitMqMessage} to a
RabbitMQ server queue
+ * or exchange.
+ *
+ * <p>As for the {@link Read}, the {@link Write} is configured with a RabbitMQ
URI.
+ *
+ * <p>For instance, you can write to an exchange (providing the exchange type):
+ *
+ * <pre>{@code
+ * pipeline
+ * .apply(...) // provide PCollection<RabbitMqMessage>
+ *
.apply(RabbitMqIO.write().withUri("amqp://user:password@localhost:5672").withExchange("EXCHANGE",
"fanout"));
+ * }</pre>
+ *
+ * <p>For instance, you can write to a queue:
+ *
+ * <pre>{@code
+ * pipeline
+ * .apply(...) // provide PCollection<RabbitMqMessage>
+ *
.apply(RabbitMqIO.write().withUri("amqp://user:password@localhost:5672").withQueue("QUEUE"));
+ *
+ * }</pre>
+ */
+@Experimental(Experimental.Kind.SOURCE_SINK)
+public class RabbitMqIO {
+ public static Read read() {
+ return new AutoValue_RabbitMqIO_Read.Builder()
+ .setQueueDeclare(false)
+ .setMaxReadTime(null)
+ .setMaxNumRecords(Long.MAX_VALUE)
+ .setUseCorrelationId(false)
+ .build();
+ }
+
+ public static Write write() {
+ return new AutoValue_RabbitMqIO_Write.Builder()
+ .setExchangeDeclare(false)
+ .setQueueDeclare(false)
+ .build();
+ }
+
+ private RabbitMqIO() {}
+
+ private static class ConnectionHandler {
+
+ private final ConnectionFactory connectionFactory;
+ private Connection connection;
+ private Channel channel;
+
+ public ConnectionHandler(String uri)
+ throws URISyntaxException, NoSuchAlgorithmException,
KeyManagementException {
+ connectionFactory = new ConnectionFactory();
+ connectionFactory.setUri(uri);
+ connectionFactory.setAutomaticRecoveryEnabled(true);
+ connectionFactory.setConnectionTimeout(60000);
+ connectionFactory.setNetworkRecoveryInterval(5000);
+ connectionFactory.setRequestedHeartbeat(60);
+ connectionFactory.setTopologyRecoveryEnabled(true);
+ connectionFactory.setRequestedChannelMax(0);
+ connectionFactory.setRequestedFrameMax(0);
+ }
+
+ public void start() throws TimeoutException, IOException {
+ connection = connectionFactory.newConnection();
+ channel = connection.createChannel();
+ if (channel == null) {
+ throw new IOException("No RabbitMQ channel available");
+ }
+ }
+
+ public Channel getChannel() {
+ return this.channel;
+ }
+
+ public void stop() throws IOException {
+ if (channel != null) {
+ try {
+ channel.close();
+ } catch (Exception e) {
+ // ignore
+ }
+ }
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ /** A {@link PTransform} to consume messages from RabbitMQ server. */
+ @AutoValue
+ public abstract static class Read extends PTransform<PBegin,
PCollection<RabbitMqMessage>> {
+ @Nullable
+ abstract String uri();
+
+ @Nullable
+ abstract String queue();
+
+ abstract boolean queueDeclare();
+
+ @Nullable
+ abstract String exchange();
+
+ @Nullable
+ abstract String exchangeType();
+
+ @Nullable
+ abstract String routingKey();
+
+ abstract boolean useCorrelationId();
+
+ abstract long maxNumRecords();
+
+ @Nullable
+ abstract Duration maxReadTime();
+
+ abstract Builder builder();
+
+ @AutoValue.Builder
+ abstract static class Builder {
+ abstract Builder setUri(String uri);
+
+ abstract Builder setQueue(String queue);
+
+ abstract Builder setQueueDeclare(boolean queueDeclare);
+
+ abstract Builder setExchange(String exchange);
+
+ abstract Builder setExchangeType(String exchangeType);
+
+ abstract Builder setRoutingKey(String routingKey);
+
+ abstract Builder setUseCorrelationId(boolean useCorrelationId);
+
+ abstract Builder setMaxNumRecords(long maxNumRecords);
+
+ abstract Builder setMaxReadTime(Duration maxReadTime);
+
+ abstract Read build();
+ }
+
+ public Read withUri(String uri) {
+ checkArgument(uri != null, "uri can not be null");
+ return builder().setUri(uri).build();
+ }
+
+ /**
+ * If you want to directly consume messages from a specific queue, you
just have to specify the
+ * queue name. Optionally, you can declare the queue using {@link
+ * RabbitMqIO.Read#withQueueDeclare(boolean)}.
+ */
+ public Read withQueue(String queue) {
+ checkArgument(queue != null, "queue can not be null");
+ return builder().setQueue(queue).build();
+ }
+
+ /**
+ * You can "force" the declaration of a queue on the RabbitMQ broker.
Exchanges and queues are
+ * the high-level building blocks of AMQP. These must be "declared" before
they can be used.
+ * Declaring either type of object simply ensures that one of that name
exists, creating it if
+ * necessary.
+ *
+ * @param queueDeclare If {@code true}, {@link RabbitMqIO} will declare
the queue. If another
+ * application declare the queue, it's not required.
+ */
+ public Read withQueueDeclare(boolean queueDeclare) {
+ return builder().setQueueDeclare(queueDeclare).build();
+ }
+
+ /**
+ * Instead of consuming messages on a specific queue, you can consume
message from a given
+ * exchange. Then you specify the exchange name, type and optionally
routing key where you want
+ * to consume messages.
+ */
+ public Read withExchange(String name, String type, String routingKey) {
+ checkArgument(name != null, "name can not be null");
+ checkArgument(type != null, "type can not be null");
+ return
builder().setExchange(name).setExchangeType(type).setRoutingKey(routingKey).build();
+ }
+
+ /**
+ * Define the max number of records received by the {@link Read}. When
this max number of
+ * records is lower than {@code Long.MAX_VALUE}, the {@link Read} will
provide a bounded {@link
+ * PCollection}.
+ */
+ public Read withMaxNumRecords(long maxNumRecords) {
+ checkArgument(maxReadTime() == null, "maxNumRecord and maxReadTime are
exclusive");
+ return builder().setMaxNumRecords(maxNumRecords).build();
+ }
+
+ /**
+ * Define the max read time (duration) while the {@link Read} will receive
messages. When this
+ * max read time is not null, the {@link Read} will provide a bounded
{@link PCollection}.
+ */
+ public Read withMaxReadTime(Duration maxReadTime) {
+ checkArgument(
+ maxNumRecords() == Long.MAX_VALUE, "maxNumRecord and maxReadTime are
exclusive");
+ return builder().setMaxReadTime(maxReadTime).build();
+ }
+
+ @Override
+ public PCollection<RabbitMqMessage> expand(PBegin input) {
+ org.apache.beam.sdk.io.Read.Unbounded<RabbitMqMessage> unbounded =
+ org.apache.beam.sdk.io.Read.from(new RabbitMQSource(this));
+
+ PTransform<PBegin, PCollection<RabbitMqMessage>> transform = unbounded;
+
+ if (maxNumRecords() < Long.MAX_VALUE || maxReadTime() != null) {
+ transform =
unbounded.withMaxReadTime(maxReadTime()).withMaxNumRecords(maxNumRecords());
+ }
+
+ return input.getPipeline().apply(transform);
+ }
+ }
+
+ static class RabbitMQSource extends UnboundedSource<RabbitMqMessage,
RabbitMQCheckpointMark> {
+ final Read spec;
+
+ RabbitMQSource(Read spec) {
+ this.spec = spec;
+ }
+
+ @Override
+ public Coder<RabbitMqMessage> getOutputCoder() {
+ return SerializableCoder.of(RabbitMqMessage.class);
+ }
+
+ @Override
+ public List<RabbitMQSource> split(int desiredNumSplits, PipelineOptions
options) {
+ // RabbitMQ uses queue, so, we can have several concurrent consumers as
source
+ List<RabbitMQSource> sources = new ArrayList<>();
+ for (int i = 0; i < desiredNumSplits; i++) {
+ sources.add(this);
+ }
+ return sources;
+ }
+
+ @Override
+ public UnboundedReader<RabbitMqMessage> createReader(
+ PipelineOptions options, RabbitMQCheckpointMark checkpointMark) throws
IOException {
+ return new UnboundedRabbitMqReader(this, checkpointMark);
+ }
+
+ @Override
+ public Coder<RabbitMQCheckpointMark> getCheckpointMarkCoder() {
+ return SerializableCoder.of(RabbitMQCheckpointMark.class);
+ }
+
+ @Override
+ public boolean requiresDeduping() {
+ return spec.useCorrelationId();
+ }
+ }
+
+ private static class RabbitMQCheckpointMark
+ implements UnboundedSource.CheckpointMark, Serializable {
+ transient Channel channel;
+ Instant oldestTimestamp;
+ final List<Long> sessionIds = new ArrayList<>();
+
+ @Override
+ public void finalizeCheckpoint() throws IOException {
+ for (Long sessionId : sessionIds) {
+ channel.basicAck(sessionId, false);
+ }
+ channel.txCommit();
+ oldestTimestamp = Instant.now();
+ sessionIds.clear();
+ }
+ }
+
+ private static class UnboundedRabbitMqReader
+ extends UnboundedSource.UnboundedReader<RabbitMqMessage> {
+ private final RabbitMQSource source;
+
+ private RabbitMqMessage current;
+ private byte[] currentRecordId;
+ private ConnectionHandler connectionHandler;
+ private QueueingConsumer consumer;
+ private Instant currentTimestamp;
+ private final RabbitMQCheckpointMark checkpointMark;
+
+ UnboundedRabbitMqReader(RabbitMQSource source, RabbitMQCheckpointMark
checkpointMark)
+ throws IOException {
+ this.source = source;
+ this.current = null;
+ this.checkpointMark = checkpointMark != null ? checkpointMark : new
RabbitMQCheckpointMark();
+ try {
+ connectionHandler = new ConnectionHandler(source.spec.uri());
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ }
+
+ @Override
+ public Instant getWatermark() {
+ return checkpointMark.oldestTimestamp;
+ }
+
+ @Override
+ public UnboundedSource.CheckpointMark getCheckpointMark() {
+ return checkpointMark;
+ }
+
+ @Override
+ public RabbitMQSource getCurrentSource() {
+ return source;
+ }
+
+ @Override
+ public byte[] getCurrentRecordId() {
+ if (current == null) {
+ throw new NoSuchElementException();
+ }
+ if (currentRecordId != null) {
+ return currentRecordId;
+ } else {
+ return "".getBytes(StandardCharsets.UTF_8);
+ }
+ }
+
+ @Override
+ public Instant getCurrentTimestamp() {
+ if (currentTimestamp == null) {
+ throw new NoSuchElementException();
+ }
+ return currentTimestamp;
+ }
+
+ @Override
+ public RabbitMqMessage getCurrent() {
+ if (current == null) {
+ throw new NoSuchElementException();
+ }
+ return current;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ try {
+ ConnectionHandler connectionHandler = new
ConnectionHandler(source.spec.uri());
+ connectionHandler.start();
+
+ Channel channel = connectionHandler.getChannel();
+
+ String queueName = source.spec.queue();
+ if (source.spec.queueDeclare()) {
+ // declare the queue (if not done by another application)
+ // channel.queueDeclare(queueName, durable, exclusive, autoDelete,
arguments);
+ channel.queueDeclare(queueName, false, false, false, null);
+ }
+ if (source.spec.exchange() != null) {
+ channel.exchangeDeclare(source.spec.exchange(),
source.spec.exchangeType());
+ if (queueName == null) {
+ queueName = channel.queueDeclare().getQueue();
+ }
+ channel.queueBind(queueName, source.spec.exchange(),
source.spec.routingKey());
+ }
+ checkpointMark.channel = channel;
+ consumer = new QueueingConsumer(channel);
+ channel.txSelect();
+ // we consume message without autoAck (we want to do the ack ourselves)
+ channel.setDefaultConsumer(consumer);
+ channel.basicConsume(queueName, false, consumer);
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ return advance();
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ try {
+ QueueingConsumer.Delivery delivery = consumer.nextDelivery(1000);
+ if (delivery == null) {
+ return false;
+ }
+ if (source.spec.useCorrelationId()) {
+ String correlationId = delivery.getProperties().getCorrelationId();
+ if (correlationId == null) {
+ throw new IOException(
+ "RabbitMqIO.Read uses message correlation ID, but received "
+ + "message has a null correlation ID");
+ }
+ currentRecordId = correlationId.getBytes(StandardCharsets.UTF_8);
+ }
+ long deliveryTag = delivery.getEnvelope().getDeliveryTag();
+ checkpointMark.sessionIds.add(deliveryTag);
+
+ current = new RabbitMqMessage(source.spec.routingKey(), delivery);
+ currentTimestamp = new
Instant(delivery.getProperties().getTimestamp());
+ if (currentTimestamp.isBefore(checkpointMark.oldestTimestamp)) {
+ checkpointMark.oldestTimestamp = currentTimestamp;
+ }
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ return true;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (connectionHandler != null) {
+ connectionHandler.stop();
+ }
+ }
+ }
+
+ /** A {@link PTransform} to publish messages to a RabbitMQ server. */
+ @AutoValue
+ public abstract static class Write
+ extends PTransform<PCollection<RabbitMqMessage>, PCollection<?>> {
+
+ @Nullable
+ abstract String uri();
+
+ @Nullable
+ abstract String exchange();
+
+ @Nullable
+ abstract String exchangeType();
+
+ abstract boolean exchangeDeclare();
+
+ @Nullable
+ abstract String queue();
+
+ abstract boolean queueDeclare();
+
+ abstract Builder builder();
+
+ @AutoValue.Builder
+ abstract static class Builder {
+ abstract Builder setUri(String uri);
+
+ abstract Builder setExchange(String exchange);
+
+ abstract Builder setExchangeType(String exchangeType);
+
+ abstract Builder setExchangeDeclare(boolean exchangeDeclare);
+
+ abstract Builder setQueue(String queue);
+
+ abstract Builder setQueueDeclare(boolean queueDeclare);
+
+ abstract Write build();
+ }
+
+ public Write withUri(String uri) {
+ checkArgument(uri != null, "uri can not be null");
+ return builder().setUri(uri).build();
+ }
+
+ /**
+ * Defines the exchange where the messages will be sent. The exchange has
to be declared. It can
+ * be done by another application or by {@link RabbitMqIO} if you define
{@code true} for {@link
+ * RabbitMqIO.Write#withExchangeDeclare(boolean)}.
+ */
+ public Write withExchange(String exchange, String exchangeType) {
+ checkArgument(exchange != null, "exchange can not be null");
+ checkArgument(exchangeType != null, "exchangeType can not be null");
+ return
builder().setExchange(exchange).setExchangeType(exchangeType).build();
+ }
+
+ /**
+ * If the exchange is not declared by another application, {@link
RabbitMqIO} can declare the
+ * exchange itself.
+ *
+ * @param exchangeDeclare {@code true} to declare the exchange, {@code
false} else.
+ */
+ public Write withExchangeDeclare(boolean exchangeDeclare) {
+ return builder().setExchangeDeclare(exchangeDeclare).build();
+ }
+
+ /**
+ * Defines the queue where the messages will be sent. The queue has to be
declared. It can be
+ * done by another application or by {@link RabbitMqIO} if you set {@link
+ * Write#withQueueDeclare} to {@code true}.
+ */
+ public Write withQueue(String queue) {
+ checkArgument(queue != null, "queue can not be null");
+ return builder().setQueue(queue).build();
+ }
+
+ /**
+ * If the queue is not declared by another application, {@link RabbitMqIO}
can declare the queue
+ * itself.
+ *
+ * @param queueDeclare {@code true} to declare the queue, {@code false}
else.
+ */
+ public Write withQueueDeclare(boolean queueDeclare) {
+ return builder().setQueueDeclare(queueDeclare).build();
+ }
+
+ @Override
+ public PCollection<?> expand(PCollection<RabbitMqMessage> input) {
+ checkArgument(
+ exchange() != null || queue() != null, "Either exchange or queue has
to be specified");
+ if (exchange() != null) {
+ checkArgument(queue() == null, "Queue can't be set in the same time as
exchange");
+ }
+ if (queue() != null) {
+ checkArgument(exchange() == null, "Exchange can't be set in the same
time as queue");
+ }
+ if (queueDeclare()) {
+ checkArgument(queue() != null, "Queue is required for the queue
declare");
+ }
+ if (exchangeDeclare()) {
+ checkArgument(exchange() != null, "Exchange is required for the
exchange declare");
+ }
+ return input.apply(ParDo.of(new WriteFn(this)));
+ }
+
+ private static class WriteFn extends DoFn<RabbitMqMessage, Void> {
+ private final Write spec;
+
+ private transient ConnectionHandler connectionHandler;
+
+ WriteFn(Write spec) {
+ this.spec = spec;
+ }
+
+ @Setup
+ public void setup() throws Exception {
+ connectionHandler = new ConnectionHandler(spec.uri());
+ connectionHandler.start();
+
+ Channel channel = connectionHandler.getChannel();
+
+ if (spec.exchange() != null && spec.exchangeDeclare()) {
+ channel.exchangeDeclare(spec.exchange(), spec.exchangeType());
+ }
+ if (spec.queue() != null && spec.queueDeclare()) {
+ channel.queueDeclare(spec.queue(), true, false, false, null);
+ }
+ }
+
+ @ProcessElement
+ public void processElement(ProcessContext c) throws IOException {
+ RabbitMqMessage message = c.element();
+ Channel channel = connectionHandler.getChannel();
+
+ if (spec.exchange() != null) {
+ channel.basicPublish(
+ spec.exchange(),
+ message.getRoutingKey(),
+ message.createProperties(),
+ message.getBody());
+ }
+ if (spec.queue() != null) {
+ channel.basicPublish(
+ "", spec.queue(), MessageProperties.PERSISTENT_TEXT_PLAIN,
message.getBody());
+ }
+ }
+
+ @Teardown
+ public void teardown() throws Exception {
+ if (connectionHandler != null) {
+ connectionHandler.stop();
+ }
+ }
+ }
+ }
+}
diff --git
a/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqMessage.java
b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqMessage.java
new file mode 100644
index 00000000000..015d1afad0f
--- /dev/null
+++
b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqMessage.java
@@ -0,0 +1,252 @@
+/*
+ * 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.rabbitmq;
+
+import com.rabbitmq.client.AMQP;
+import com.rabbitmq.client.QueueingConsumer;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.annotation.Nullable;
+
+/**
+ * It contains the message payload, and additional metadata like routing key
or attributes. The main
+ * reason of this class is that AMQP.BasicProperties doesn't provide a
serializable public API.
+ */
+public class RabbitMqMessage implements Serializable {
+
+ @Nullable private final String routingKey;
+ private final byte[] body;
+ private final String contentType;
+ private final String contentEncoding;
+ private final Map<String, Object> headers;
+ private final Integer deliveryMode;
+ private final Integer priority;
+ @Nullable private final String correlationId;
+ @Nullable private final String replyTo;
+ private final String expiration;
+ private final String messageId;
+ private final Date timestamp;
+ @Nullable private final String type;
+ @Nullable private final String userId;
+ @Nullable private final String appId;
+ @Nullable private final String clusterId;
+
+ public RabbitMqMessage(byte[] body) {
+ this.body = body;
+ routingKey = "";
+ contentType = null;
+ contentEncoding = null;
+ headers = new HashMap<>();
+ deliveryMode = 1;
+ priority = 1;
+ correlationId = null;
+ replyTo = null;
+ expiration = null;
+ messageId = null;
+ timestamp = new Date();
+ type = null;
+ userId = null;
+ appId = null;
+ clusterId = null;
+ }
+
+ public RabbitMqMessage(String routingKey, QueueingConsumer.Delivery
delivery) {
+ this.routingKey = routingKey;
+ body = delivery.getBody();
+ contentType = delivery.getProperties().getContentType();
+ contentEncoding = delivery.getProperties().getContentEncoding();
+ headers = delivery.getProperties().getHeaders();
+ deliveryMode = delivery.getProperties().getDeliveryMode();
+ priority = delivery.getProperties().getPriority();
+ correlationId = delivery.getProperties().getCorrelationId();
+ replyTo = delivery.getProperties().getReplyTo();
+ expiration = delivery.getProperties().getExpiration();
+ messageId = delivery.getProperties().getMessageId();
+ timestamp = delivery.getProperties().getTimestamp();
+ type = delivery.getProperties().getType();
+ userId = delivery.getProperties().getUserId();
+ appId = delivery.getProperties().getAppId();
+ clusterId = delivery.getProperties().getClusterId();
+ }
+
+ public RabbitMqMessage(
+ String routingKey,
+ byte[] body,
+ String contentType,
+ String contentEncoding,
+ Map<String, Object> headers,
+ Integer deliveryMode,
+ Integer priority,
+ String correlationId,
+ String replyTo,
+ String expiration,
+ String messageId,
+ Date timestamp,
+ String type,
+ String userId,
+ String appId,
+ String clusterId) {
+ this.routingKey = routingKey;
+ this.body = body;
+ this.contentType = contentType;
+ this.contentEncoding = contentEncoding;
+ this.headers = headers;
+ this.deliveryMode = deliveryMode;
+ this.priority = priority;
+ this.correlationId = correlationId;
+ this.replyTo = replyTo;
+ this.expiration = expiration;
+ this.messageId = messageId;
+ this.timestamp = timestamp;
+ this.type = type;
+ this.userId = userId;
+ this.appId = appId;
+ this.clusterId = clusterId;
+ }
+
+ public String getRoutingKey() {
+ return routingKey;
+ }
+
+ public byte[] getBody() {
+ return body;
+ }
+
+ public String getContentType() {
+ return contentType;
+ }
+
+ public String getContentEncoding() {
+ return contentEncoding;
+ }
+
+ public Map<String, Object> getHeaders() {
+ return headers;
+ }
+
+ public Integer getDeliveryMode() {
+ return deliveryMode;
+ }
+
+ public Integer getPriority() {
+ return priority;
+ }
+
+ public String getCorrelationId() {
+ return correlationId;
+ }
+
+ public String getReplyTo() {
+ return replyTo;
+ }
+
+ public String getExpiration() {
+ return expiration;
+ }
+
+ public String getMessageId() {
+ return messageId;
+ }
+
+ public Date getTimestamp() {
+ return timestamp;
+ }
+
+ public String getType() {
+ return type;
+ }
+
+ public String getUserId() {
+ return userId;
+ }
+
+ public String getAppId() {
+ return appId;
+ }
+
+ public String getClusterId() {
+ return clusterId;
+ }
+
+ public AMQP.BasicProperties createProperties() {
+ return new AMQP.BasicProperties()
+ .builder()
+ .contentType(contentType)
+ .contentEncoding(contentEncoding)
+ .headers(headers)
+ .deliveryMode(deliveryMode)
+ .priority(priority)
+ .correlationId(correlationId)
+ .replyTo(replyTo)
+ .expiration(expiration)
+ .messageId(messageId)
+ .timestamp(timestamp)
+ .type(type)
+ .userId(userId)
+ .appId(appId)
+ .clusterId(clusterId)
+ .build();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ routingKey,
+ Arrays.hashCode(body),
+ contentType,
+ contentEncoding,
+ deliveryMode,
+ priority,
+ correlationId,
+ replyTo,
+ expiration,
+ messageId,
+ timestamp,
+ type,
+ userId,
+ appId,
+ clusterId);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof RabbitMqMessage) {
+ RabbitMqMessage other = (RabbitMqMessage) obj;
+ return Objects.equals(routingKey, other.routingKey)
+ && Arrays.equals(body, other.body)
+ && Objects.equals(contentType, other.contentType)
+ && Objects.equals(contentEncoding, other.contentEncoding)
+ && Objects.equals(deliveryMode, other.deliveryMode)
+ && Objects.equals(priority, other.priority)
+ && Objects.equals(correlationId, other.correlationId)
+ && Objects.equals(replyTo, other.replyTo)
+ && Objects.equals(expiration, other.expiration)
+ && Objects.equals(messageId, other.messageId)
+ && Objects.equals(timestamp, other.timestamp)
+ && Objects.equals(type, other.type)
+ && Objects.equals(userId, other.userId)
+ && Objects.equals(appId, other.appId)
+ && Objects.equals(clusterId, other.clusterId);
+ }
+ return false;
+ }
+}
diff --git
a/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/package-info.java
b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/package-info.java
new file mode 100644
index 00000000000..c2f796ffc26
--- /dev/null
+++
b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+/** Transforms for reading and writing from RabbitMQ. */
+package org.apache.beam.sdk.io.rabbitmq;
diff --git
a/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIOTest.java
b/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIOTest.java
new file mode 100644
index 00000000000..d403c38fdcd
--- /dev/null
+++
b/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIOTest.java
@@ -0,0 +1,315 @@
+/*
+ * 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.rabbitmq;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.rabbitmq.client.AMQP;
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.Consumer;
+import com.rabbitmq.client.DefaultConsumer;
+import com.rabbitmq.client.Envelope;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.ServerSocket;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.qpid.server.Broker;
+import org.apache.qpid.server.BrokerOptions;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Test of {@link RabbitMqIO}. */
+@RunWith(JUnit4.class)
+public class RabbitMqIOTest implements Serializable {
+ private static final Logger LOG =
LoggerFactory.getLogger(RabbitMqIOTest.class);
+
+ private static int port;
+ @ClassRule public static TemporaryFolder temporaryFolder = new
TemporaryFolder();
+
+ @Rule public transient TestPipeline p = TestPipeline.create();
+
+ private static transient Broker broker;
+
+ @BeforeClass
+ public static void startBroker() throws Exception {
+ try (ServerSocket serverSocket = new ServerSocket(0)) {
+ port = serverSocket.getLocalPort();
+ }
+
+ System.setProperty("derby.stream.error.field", "MyApp.DEV_NULL");
+ broker = new Broker();
+ BrokerOptions options = new BrokerOptions();
+ options.setConfigProperty(BrokerOptions.QPID_AMQP_PORT,
String.valueOf(port));
+ options.setConfigProperty(BrokerOptions.QPID_WORK_DIR,
temporaryFolder.newFolder().toString());
+ options.setConfigProperty(BrokerOptions.QPID_HOME_DIR, "src/test/qpid");
+ broker.startup(options);
+ }
+
+ @AfterClass
+ public static void stopBroker() {
+ broker.shutdown();
+ }
+
+ @Test
+ public void testReadQueue() throws Exception {
+ final int maxNumRecords = 10;
+ PCollection<RabbitMqMessage> raw =
+ p.apply(
+ RabbitMqIO.read()
+ .withUri("amqp://guest:guest@localhost:" + port)
+ .withQueue("READ")
+ .withMaxNumRecords(maxNumRecords));
+ PCollection<String> output =
+ raw.apply(
+ MapElements.into(TypeDescriptors.strings())
+ .via(
+ (RabbitMqMessage message) ->
+ new String(message.getBody(),
StandardCharsets.UTF_8)));
+
+ List<String> records =
+ generateRecords(maxNumRecords)
+ .stream()
+ .map(record -> new String(record, StandardCharsets.UTF_8))
+ .collect(Collectors.toList());
+ PAssert.that(output).containsInAnyOrder(records);
+
+ ConnectionFactory connectionFactory = new ConnectionFactory();
+ connectionFactory.setUri("amqp://guest:guest@localhost:" + port);
+ Connection connection = null;
+ Channel channel = null;
+ try {
+ connection = connectionFactory.newConnection();
+ channel = connection.createChannel();
+ channel.queueDeclare("READ", false, false, false, null);
+ for (String record : records) {
+ channel.basicPublish("", "READ", null,
record.getBytes(StandardCharsets.UTF_8));
+ }
+
+ p.run();
+ } finally {
+ if (channel != null) {
+ channel.close();
+ }
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test(timeout = 60 * 1000)
+ public void testReadExchange() throws Exception {
+ final int maxNumRecords = 10;
+ PCollection<RabbitMqMessage> raw =
+ p.apply(
+ RabbitMqIO.read()
+ .withUri("amqp://guest:guest@localhost:" + port)
+ .withExchange("READEXCHANGE", "fanout", "test")
+ .withMaxNumRecords(maxNumRecords));
+ PCollection<String> output =
+ raw.apply(
+ MapElements.into(TypeDescriptors.strings())
+ .via(
+ (RabbitMqMessage message) ->
+ new String(message.getBody(),
StandardCharsets.UTF_8)));
+
+ List<String> records =
+ generateRecords(maxNumRecords)
+ .stream()
+ .map(record -> new String(record, StandardCharsets.UTF_8))
+ .collect(Collectors.toList());
+ PAssert.that(output).containsInAnyOrder(records);
+
+ ConnectionFactory connectionFactory = new ConnectionFactory();
+ connectionFactory.setUri("amqp://guest:guest@localhost:" + port);
+ Connection connection = null;
+ Channel channel = null;
+ try {
+ connection = connectionFactory.newConnection();
+ channel = connection.createChannel();
+ channel.exchangeDeclare("READEXCHANGE", "fanout");
+ Channel finalChannel = channel;
+ Thread publisher =
+ new Thread(
+ () -> {
+ try {
+ Thread.sleep(5000);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ for (int i = 0; i < maxNumRecords; i++) {
+ try {
+ finalChannel.basicPublish(
+ "READEXCHANGE",
+ "test",
+ null,
+ ("Test " + i).getBytes(StandardCharsets.UTF_8));
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+ });
+ publisher.start();
+ p.run();
+ publisher.join();
+ } finally {
+ if (channel != null) {
+ channel.close();
+ }
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void testWriteQueue() throws Exception {
+ final int maxNumRecords = 1000;
+ List<RabbitMqMessage> data =
+ generateRecords(maxNumRecords)
+ .stream()
+ .map(bytes -> new RabbitMqMessage(bytes))
+ .collect(Collectors.toList());
+ p.apply(Create.of(data))
+ .apply(
+ RabbitMqIO.write().withUri("amqp://guest:guest@localhost:" +
port).withQueue("TEST"));
+
+ final List<String> received = new ArrayList<>();
+ ConnectionFactory connectionFactory = new ConnectionFactory();
+ connectionFactory.setUri("amqp://guest:guest@localhost:" + port);
+ Connection connection = null;
+ Channel channel = null;
+ try {
+ connection = connectionFactory.newConnection();
+ channel = connection.createChannel();
+ channel.queueDeclare("TEST", true, false, false, null);
+ Consumer consumer = new TestConsumer(channel, received);
+ channel.basicConsume("TEST", true, consumer);
+
+ p.run();
+
+ while (received.size() < maxNumRecords) {
+ Thread.sleep(500);
+ }
+
+ assertEquals(maxNumRecords, received.size());
+ for (int i = 0; i < maxNumRecords; i++) {
+ assertTrue(received.contains("Test " + i));
+ }
+ } finally {
+ if (channel != null) {
+ channel.close();
+ }
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void testWriteExchange() throws Exception {
+ final int maxNumRecords = 1000;
+ List<RabbitMqMessage> data =
+ generateRecords(maxNumRecords)
+ .stream()
+ .map(bytes -> new RabbitMqMessage(bytes))
+ .collect(Collectors.toList());
+ p.apply(Create.of(data))
+ .apply(
+ RabbitMqIO.write()
+ .withUri("amqp://guest:guest@localhost:" + port)
+ .withExchange("WRITE", "fanout"));
+
+ final List<String> received = new ArrayList<>();
+ ConnectionFactory connectionFactory = new ConnectionFactory();
+ connectionFactory.setUri("amqp://guest:guest@localhost:" + port);
+ Connection connection = null;
+ Channel channel = null;
+ try {
+ connection = connectionFactory.newConnection();
+ channel = connection.createChannel();
+ channel.exchangeDeclare("WRITE", "fanout");
+ String queueName = channel.queueDeclare().getQueue();
+ channel.queueBind(queueName, "WRITE", "");
+ Consumer consumer = new TestConsumer(channel, received);
+ channel.basicConsume(queueName, true, consumer);
+
+ p.run();
+
+ while (received.size() < maxNumRecords) {
+ Thread.sleep(500);
+ }
+
+ assertEquals(maxNumRecords, received.size());
+ for (int i = 0; i < maxNumRecords; i++) {
+ assertTrue(received.contains("Test " + i));
+ }
+ } finally {
+ if (channel != null) {
+ channel.close();
+ }
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private static List<byte[]> generateRecords(int maxNumRecords) {
+ return IntStream.range(0, maxNumRecords)
+ .mapToObj(i -> ("Test " + i).getBytes(StandardCharsets.UTF_8))
+ .collect(Collectors.toList());
+ }
+
+ private static class TestConsumer extends DefaultConsumer {
+
+ private final List<String> received;
+
+ public TestConsumer(Channel channel, List<String> received) {
+ super(channel);
+ this.received = received;
+ }
+
+ @Override
+ public void handleDelivery(
+ String consumerTag, Envelope envelope, AMQP.BasicProperties
properties, byte[] body)
+ throws IOException {
+ String message = new String(body, "UTF-8");
+ received.add(message);
+ }
+ }
+}
diff --git a/sdks/java/io/rabbitmq/src/test/qpid/etc/passwd
b/sdks/java/io/rabbitmq/src/test/qpid/etc/passwd
new file mode 100644
index 00000000000..966a16153d6
--- /dev/null
+++ b/sdks/java/io/rabbitmq/src/test/qpid/etc/passwd
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+guest:guest
diff --git a/settings.gradle b/settings.gradle
index 115cb9ce5b1..be73374a341 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -152,6 +152,8 @@ include "beam-sdks-java-io-mqtt"
project(":beam-sdks-java-io-mqtt").dir = file("sdks/java/io/mqtt")
include "beam-sdks-java-io-parquet"
project(":beam-sdks-java-io-parquet").dir = file("sdks/java/io/parquet")
+include "beam-sdks-java-io-rabbitmq"
+project(":beam-sdks-java-io-rabbitmq").dir = file("sdks/java/io/rabbitmq")
include "beam-sdks-java-io-redis"
project(":beam-sdks-java-io-redis").dir = file("sdks/java/io/redis")
include "beam-sdks-java-io-solr"
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 158571)
Time Spent: 13h 50m (was: 13h 40m)
> Create RabbitMqIO
> -----------------
>
> Key: BEAM-1240
> URL: https://issues.apache.org/jira/browse/BEAM-1240
> Project: Beam
> Issue Type: New Feature
> Components: io-ideas
> Reporter: Jean-Baptiste Onofré
> Assignee: Jean-Baptiste Onofré
> Priority: Major
> Fix For: 2.8.0
>
> Time Spent: 13h 50m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)