[ 
https://issues.apache.org/jira/browse/BEAM-1240?focusedWorklogId=125187&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-125187
 ]

ASF GitHub Bot logged work on BEAM-1240:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 19/Jul/18 19:03
            Start Date: 19/Jul/18 19:03
    Worklog Time Spent: 10m 
      Work Description: jkff commented on a change in pull request #1729: 
[BEAM-1240] Create RabbitMqIO
URL: https://github.com/apache/beam/pull/1729#discussion_r203836361
 
 

 ##########
 File path: 
sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
 ##########
 @@ -0,0 +1,530 @@
+/*
+ * 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.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 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.apache.beam.sdk.values.PDone;
+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[]}).
+ *
+ * <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:
+ *
+ * <pre>{@code
+ *
+ *  pipeline.apply(
+ *    
RabbitMqIO.read().withUri("amqp://user:password@localhost:5672").withQueue("QUEUE")
+ *
+ * }</pre>
+ *
+ * <h3>Publishing messages to RabbitMQ server</h3>
+ *
+ * <p>{@link RabbitMqIO} {@link Write} can send {@code byte[]} to a RabbitMQ 
server queue.
+ *
+ * <p>As for the {@link Read}, the {@link Write} is configured with a RabbitMQ 
URI.
+ *
+ * <p>For instance:
+ *
+ * <pre>{@code
+ *
+ *  pipeline
+ *    .apply(...) // provide PCollection<byte[]>
+ *    
.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).build();
+  }
+
+  private RabbitMqIO() {
+  }
+
+  private static ConnectionFactory createConnectionFactory(String uri) throws 
URISyntaxException,
+          NoSuchAlgorithmException, KeyManagementException {
+    ConnectionFactory 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);
+    return connectionFactory;
+  }
+
+  /**
+   * 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 =
 
 Review comment:
   Still relevant

----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 125187)
    Time Spent: 8.5h  (was: 8h 20m)

> 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.6.0
>
>          Time Spent: 8.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to