[
https://issues.apache.org/jira/browse/BEAM-1240?focusedWorklogId=79096&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-79096
]
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_r173574242
##########
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);
Review comment:
Same comments about these parameters as in the JDBC PR. Do not mirror every
parameter of the underlying library - expose only parameters that are
absolutely essential, and have predictable impact regardless of the
implementation of RabbitMqIO or the runner behavior.
----------------------------------------------------------------
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: 79096)
Time Spent: 40m (was: 0.5h)
> 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: 40m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)