You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/03/09 22:03:01 UTC

[jira] [Work logged] (BEAM-1240) Create RabbitMqIO

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

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

                Author: ASF GitHub Bot
            Created on: 09/Mar/18 22:02
            Start Date: 09/Mar/18 22:02
    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_r173574801
 
 

 ##########
 File path: sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
 ##########
 @@ -0,0 +1,711 @@
+/*
+ * 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.AMQP;
+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.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
+ * {@link ConnectionConfig}.
+ *
+ * <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()
+        .setConnectionConfig(ConnectionConfig.create()).setQueueDeclare(false)
+        .setMaxReadTime(null).setMaxNumRecords(Long.MAX_VALUE).setUseCorrelationId(false).build();
+  }
+
+  public static Write write() {
+    return new AutoValue_RabbitMqIO_Write.Builder()
+        .setConnectionConfig(ConnectionConfig.create())
+        .setExchangeDeclare(false).build();
+  }
+
+  private RabbitMqIO() {
+  }
+
+  /**
+   * Describe a connection configuration to a RabbitMQ server.
+   */
+  @AutoValue
+  public abstract static class ConnectionConfig implements Serializable {
+
+    @Nullable abstract String uri();
+
+    abstract int networkRecoveryInterval();
+    abstract boolean automaticRecovery();
+    abstract boolean topologyRecovery();
+
+    abstract int connectionTimeout();
+    abstract int requestedChannelMax();
+    abstract int requestedFrameMax();
+    abstract int requestedHeartbeat();
+
+    abstract Builder builder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setUri(String uri);
+      abstract Builder setNetworkRecoveryInterval(int networkRecoveryInterval);
+      abstract Builder setAutomaticRecovery(boolean automaticRecovery);
+      abstract Builder setTopologyRecovery(boolean topologyRecovery);
+      abstract Builder setConnectionTimeout(int connectionTimeout);
+      abstract Builder setRequestedChannelMax(int requestedChannelMax);
+      abstract Builder setRequestedFrameMax(int requestedFrameMax);
+      abstract Builder setRequestedHeartbeat(int requestedHeartbeat);
+      abstract ConnectionConfig build();
+    }
+
+    public static ConnectionConfig create() {
+      return new AutoValue_RabbitMqIO_ConnectionConfig.Builder()
+          .setUri("amqp://localhost:5672")
+          .setAutomaticRecovery(true)
+          .setTopologyRecovery(true)
+          .setConnectionTimeout(60000)
+          .setRequestedChannelMax(0)
+          .setRequestedFrameMax(0)
+          .setRequestedHeartbeat(60)
+          .setNetworkRecoveryInterval(5000)
+          .build();
+    }
+
+    /**
+     * Create a RabbitMQ connection configuration with broker URI and a queue name.
+     *
+     * @param uri The RabbitMQ server URI.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public static ConnectionConfig create(String uri) {
+      checkArgument(uri != null, "uri can not be null");
+      return new AutoValue_RabbitMqIO_ConnectionConfig.Builder()
+          .setUri(uri)
+          .setAutomaticRecovery(true)
+          .setTopologyRecovery(true)
+          .setConnectionTimeout(60000)
+          .setRequestedChannelMax(0)
+          .setRequestedFrameMax(0)
+          .setRequestedHeartbeat(60)
+          .setNetworkRecoveryInterval(5000)
+          .build();
+    }
+
+    /**
+     * Define the RabbitMQ URI.
+     *
+     * @param uri The RabbitMQ URI.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withUri(String uri) {
+      checkArgument(uri != null, "uri can not be null");
+      return builder().setUri(uri).build();
+    }
+
+    /**
+     * Define the RabbitMQ connection network recovery interval.
+     *
+     * @param networkRecoveryInterval The network recovery interval (in ms).
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withNetworkRecoveryInterval(int networkRecoveryInterval) {
+      checkArgument(networkRecoveryInterval >= 0,
+          "networkRecoveryInterval has to be positive or 0");
+      return builder().setNetworkRecoveryInterval(networkRecoveryInterval).build();
+    }
+
+    /**
+     * Define the RabbitMQ connection automatic recovery.
+     *
+     * @param automaticRecovery True to enable automatic recovery on the RabbitMQ connection,
+     *                          false else.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withAutomaticRecovery(boolean automaticRecovery) {
+      return builder().setAutomaticRecovery(automaticRecovery).build();
+    }
+
+    /**
+     * Define the RabbitMQ connection topology recovery.
+     *
+     * @param topologyRecovery True to enable topology recovery on the RabbitMQ connection, false
+     *                         else.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withTopologyRecovery(boolean topologyRecovery) {
+      return builder().setTopologyRecovery(topologyRecovery).build();
+    }
+
+    /**
+     * Define the RabbitMQ connection timeout.
+     *
+     * @param connectionTimeout The connection timeout in ms.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withConnectionTimeout(int connectionTimeout) {
+      checkArgument(connectionTimeout >= 0,
+          "connectionTimeout has to be positive or 0");
+      return builder().setConnectionTimeout(connectionTimeout).build();
+    }
+
+    /**
+     * Define the RabbitMQ requested channel max number.
+     *
+     * @param requestedChannelMax The max number of requested channel.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withRequestedChannelMax(int requestedChannelMax) {
+      checkArgument(requestedChannelMax >= 0,
+          "requestedChannelMax has to be positive or 0");
+      return builder().setRequestedChannelMax(requestedChannelMax).build();
+    }
+
+    /**
+     * Define the RabbitMQ requested frame max number.
+     *
+     * @param requestedFrameMax The max number of requested frame.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withRequestedFrameMax(int requestedFrameMax) {
+      checkArgument(requestedFrameMax >= 0,
+          "requestedFrameMax has to be positive or 0");
+      return builder().setRequestedFrameMax(requestedFrameMax).build();
+    }
+
+    /**
+     * Define the RabbitMQ requested heartbeat number.
+     *
+     * @param requestedHeartbeat The number of requested heartbeat to perform.
+     * @return The corresponding {@link ConnectionConfig}.
+     */
+    public ConnectionConfig withRequestedHeartbeat(int requestedHeartbeat) {
+      checkArgument(requestedHeartbeat >= 0,
+          "requestedHeartbeat has to be positive or 0");
+      return builder().setRequestedHeartbeat(requestedHeartbeat).build();
+    }
+
+    ConnectionFactory createConnectionFactory() throws URISyntaxException,
+        NoSuchAlgorithmException, KeyManagementException {
+      ConnectionFactory connectionFactory = new ConnectionFactory();
+      connectionFactory.setUri(uri());
+
+      connectionFactory.setAutomaticRecoveryEnabled(automaticRecovery());
+      connectionFactory.setConnectionTimeout(connectionTimeout());
+      connectionFactory.setNetworkRecoveryInterval(networkRecoveryInterval());
+      connectionFactory.setRequestedHeartbeat(requestedHeartbeat());
+      connectionFactory.setTopologyRecoveryEnabled(topologyRecovery());
+      connectionFactory.setRequestedChannelMax(requestedChannelMax());
+      connectionFactory.setRequestedFrameMax(requestedFrameMax());
+
+      return connectionFactory;
+    }
+
+  }
+
+  /**
+   * A {@link PTransform} to consume messages from RabbitMQ server.
+   */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, PCollection<RabbitMqMessage>> {
+
+    @Nullable abstract ConnectionConfig connectionConfig();
+    @Nullable abstract String queue();
+    abstract boolean queueDeclare();
+    @Nullable abstract String exchange();
+    @Nullable abstract String exchangeType();
+    @Nullable abstract String routingKey();
+    @Nullable abstract Boolean useCorrelationId();
+    abstract long maxNumRecords();
+    @Nullable abstract Duration maxReadTime();
+
+    abstract Builder builder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setConnectionConfig(ConnectionConfig connectionConfig);
+      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 withConnectionConfig(ConnectionConfig connectionConfig) {
+      checkArgument(connectionConfig != null, "connectionConfig can not be null");
+      return builder().setConnectionConfig(connectionConfig).build();
+    }
+
+    public Read withUri(String uri) {
+      checkArgument(uri != null, "uri can not be null");
+      return builder().setConnectionConfig(connectionConfig().withUri(uri)).build();
+    }
+
+    public Read withQueue(String queue) {
+      checkArgument(queue != null, "queue can not be null");
+      return builder().setQueue(queue).build();
+    }
+
+    public Read withQueueDeclare(boolean queueDeclare) {
+      return builder().setQueueDeclare(queueDeclare).build();
+    }
+
+    public Read withExchange(String name, String type, String routingKey) {
 
 Review comment:
   What does this do and when does it or does it not need to be set?

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


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

    Worklog Id:     (was: 79100)
    Time Spent: 1h 20m  (was: 1h 10m)

> 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
>          Time Spent: 1h 20m
>  Remaining Estimate: 0h
>




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