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

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

                Author: ASF GitHub Bot
            Created on: 22/Aug/18 20:16
            Start Date: 22/Aug/18 20:16
    Worklog Time Spent: 10m 
      Work Description: iemejia closed pull request #6101: [BEAM-4828] Add 
Amazon SqsIO
URL: https://github.com/apache/beam/pull/6101
 
 
   

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/amazon-web-services/build.gradle 
b/sdks/java/io/amazon-web-services/build.gradle
index 973d115dbf2..3ea755966cd 100644
--- a/sdks/java/io/amazon-web-services/build.gradle
+++ b/sdks/java/io/amazon-web-services/build.gradle
@@ -1,3 +1,5 @@
+import groovy.json.JsonOutput
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -29,6 +31,7 @@ dependencies {
   shadow project(path: ":beam-sdks-java-core", configuration: "shadow")
   shadow "com.amazonaws:aws-java-sdk-core:$aws_java_sdk_version"
   shadow "com.amazonaws:aws-java-sdk-s3:$aws_java_sdk_version"
+  shadow "com.amazonaws:aws-java-sdk-sqs:$aws_java_sdk_version"
   shadow library.java.jackson_core
   shadow library.java.jackson_annotations
   shadow library.java.jackson_databind
@@ -42,5 +45,13 @@ dependencies {
   shadowTest library.java.mockito_core
   shadowTest library.java.junit
   shadowTest library.java.slf4j_jdk14
-  testCompile "io.findify:s3mock_2.11:0.2.4"
+  testCompile "io.findify:s3mock_2.12:0.2.4"
+  shadowTest group: 'org.elasticmq', name: 'elasticmq-rest-sqs_2.12', version: 
'0.14.1'
+}
+
+test {
+  systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
+      '--awsRegion=us-west-2',
+      '--awsCredentialsProvider={"@type" : "AWSStaticCredentialsProvider", 
"awsAccessKeyId" : "key_id_value","awsSecretKey" : "secret_value"}'
+  ])
 }
diff --git 
a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsCheckpointMark.java
 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsCheckpointMark.java
new file mode 100644
index 00000000000..d9a67c066d7
--- /dev/null
+++ 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsCheckpointMark.java
@@ -0,0 +1,65 @@
+/*
+ * 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.aws.sqs;
+
+import com.amazonaws.services.sqs.model.Message;
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.beam.sdk.io.UnboundedSource;
+
+class SqsCheckpointMark implements UnboundedSource.CheckpointMark, 
Serializable {
+
+  private final List<Message> messagesToDelete;
+  private final transient Optional<SqsUnboundedReader> reader;
+
+  public SqsCheckpointMark(SqsUnboundedReader reader, Collection<Message> 
messagesToDelete) {
+    this.reader = Optional.of(reader);
+    this.messagesToDelete = ImmutableList.copyOf(messagesToDelete);
+    ;
+  }
+
+  @Override
+  public void finalizeCheckpoint() {
+    reader.ifPresent(r -> r.delete(messagesToDelete));
+  }
+
+  List<Message> getMessagesToDelete() {
+    return messagesToDelete;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    SqsCheckpointMark that = (SqsCheckpointMark) o;
+    return Objects.equal(messagesToDelete, that.messagesToDelete);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(messagesToDelete);
+  }
+}
diff --git 
a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsConfiguration.java
 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsConfiguration.java
new file mode 100644
index 00000000000..f1128b9edba
--- /dev/null
+++ 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsConfiguration.java
@@ -0,0 +1,78 @@
+/*
+ * 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.aws.sqs;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.Serializable;
+import org.apache.beam.sdk.io.aws.options.AwsModule;
+import org.apache.beam.sdk.io.aws.options.AwsOptions;
+
+class SqsConfiguration implements Serializable {
+
+  private String awsRegion;
+  private String awsCredentialsProviderString;
+  private String awsClientConfigurationString;
+
+  public SqsConfiguration(AwsOptions awsOptions) {
+    ObjectMapper om = new ObjectMapper();
+    om.registerModule(new AwsModule());
+    try {
+      this.awsCredentialsProviderString =
+          om.writeValueAsString(awsOptions.getAwsCredentialsProvider());
+    } catch (JsonProcessingException e) {
+      this.awsCredentialsProviderString = null;
+    }
+
+    try {
+      this.awsClientConfigurationString =
+          om.writeValueAsString(awsOptions.getClientConfiguration());
+    } catch (JsonProcessingException e) {
+      this.awsClientConfigurationString = null;
+    }
+
+    this.awsRegion = awsOptions.getAwsRegion();
+  }
+
+  public AWSCredentialsProvider getAwsCredentialsProvider() {
+    ObjectMapper om = new ObjectMapper();
+    om.registerModule(new AwsModule());
+    try {
+      return om.readValue(awsCredentialsProviderString, 
AWSCredentialsProvider.class);
+    } catch (IOException e) {
+      return null;
+    }
+  }
+
+  public ClientConfiguration getClientConfiguration() {
+    ObjectMapper om = new ObjectMapper();
+    om.registerModule(new AwsModule());
+    try {
+      return om.readValue(awsClientConfigurationString, 
ClientConfiguration.class);
+    } catch (IOException e) {
+      return null;
+    }
+  }
+
+  public String getAwsRegion() {
+    return awsRegion;
+  }
+}
diff --git 
a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
new file mode 100644
index 00000000000..cb2b0c35be4
--- /dev/null
+++ 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
@@ -0,0 +1,216 @@
+/*
+ * 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.aws.sqs;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.amazonaws.services.sqs.AmazonSQS;
+import com.amazonaws.services.sqs.AmazonSQSClientBuilder;
+import com.amazonaws.services.sqs.model.Message;
+import com.amazonaws.services.sqs.model.SendMessageRequest;
+import com.google.auto.value.AutoValue;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.aws.options.AwsOptions;
+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;
+
+/**
+ * An unbounded source for Amazon Simple Queue Service (SQS).
+ *
+ * <h3>Reading from an SQS queue</h3>
+ *
+ * <p>The {@link SqsIO} {@link Read} returns an unbounded {@link PCollection} 
of {@link
+ * com.amazonaws.services.sqs.model.Message} containing the received messages. 
Note: This source
+ * does not currently advance the watermark when no new messages are received.
+ *
+ * <p>To configure an SQS source, you have to provide the queueUrl to connect 
to. The following
+ * example illustrates how to configure the source:
+ *
+ * <pre>{@code
+ * pipeline.apply(SqsIO.read().withQueueUrl(queueUrl))
+ * }</pre>
+ *
+ * <h3>Writing to an SQS queue</h3>
+ *
+ * <p>The following example illustrates how to use the sink:
+ *
+ * <pre>{@code
+ * pipeline
+ *   .apply(...) // returns PCollection<SendMessageRequest>
+ *   .apply(SqsIO.write())
+ * }</pre>
+ *
+ * <h3>Additional Configuration</h3>
+ *
+ * <p>Additional configuration can be provided via {@link AwsOptions} from 
command line args or in
+ * code. For example, if you wanted to provide a secret access key via code:
+ *
+ * <pre>{@code
+ * PipelineOptions pipelineOptions = 
PipelineOptionsFactory.fromArgs(args).withValidation().create();
+ * AwsOptions awsOptions = pipelineOptions.as(AwsOptions.class);
+ * BasicAWSCredentials awsCreds = new BasicAWSCredentials("accesskey", 
"secretkey");
+ * awsOptions.setAwsCredentialsProvider(new 
AWSStaticCredentialsProvider(awsCreds));
+ * Pipeline pipeline = Pipeline.create(options);
+ * }</pre>
+ *
+ * <p>For more information on the available options see {@link AwsOptions}.
+ */
+@Experimental(Experimental.Kind.SOURCE_SINK)
+public class SqsIO {
+
+  public static Read read() {
+    return new 
AutoValue_SqsIO_Read.Builder().setMaxNumRecords(Long.MAX_VALUE).build();
+  }
+
+  public static Write write() {
+    return new AutoValue_SqsIO_Write.Builder().build();
+  }
+
+  private SqsIO() {}
+
+  /**
+   * A {@link PTransform} to read/receive messages from SQS. See {@link SqsIO} 
for more information
+   * on usage and configuration.
+   */
+  @AutoValue
+  public abstract static class Read extends PTransform<PBegin, 
PCollection<Message>> {
+
+    @Nullable
+    abstract String queueUrl();
+
+    abstract long maxNumRecords();
+
+    @Nullable
+    abstract Duration maxReadTime();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setQueueUrl(String queueUrl);
+
+      abstract Builder setMaxNumRecords(long maxNumRecords);
+
+      abstract Builder setMaxReadTime(Duration maxReadTime);
+
+      abstract Read build();
+    }
+
+    /**
+     * Define the max number of records received by the {@link Read}. When the 
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) {
+      return toBuilder().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) {
+      return toBuilder().setMaxReadTime(maxReadTime).build();
+    }
+
+    /** Define the queueUrl used by the {@link Read} to receive messages from 
SQS. */
+    public Read withQueueUrl(String queueUrl) {
+      checkArgument(queueUrl != null, "queueUrl can not be null");
+      checkArgument(!queueUrl.isEmpty(), "queueUrl can not be empty");
+      return toBuilder().setQueueUrl(queueUrl).build();
+    }
+
+    @Override
+    public PCollection<Message> expand(PBegin input) {
+
+      org.apache.beam.sdk.io.Read.Unbounded<Message> unbounded =
+          org.apache.beam.sdk.io.Read.from(
+              new SqsUnboundedSource(
+                  this,
+                  new 
SqsConfiguration(input.getPipeline().getOptions().as(AwsOptions.class))));
+
+      PTransform<PBegin, PCollection<Message>> transform = unbounded;
+
+      if (maxNumRecords() < Long.MAX_VALUE || maxReadTime() != null) {
+        transform = 
unbounded.withMaxReadTime(maxReadTime()).withMaxNumRecords(maxNumRecords());
+      }
+
+      return input.getPipeline().apply(transform);
+    }
+  }
+
+  /**
+   * A {@link PTransform} to send messages to SQS. See {@link SqsIO} for more 
information on usage
+   * and configuration.
+   */
+  @AutoValue
+  public abstract static class Write extends 
PTransform<PCollection<SendMessageRequest>, PDone> {
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Write build();
+    }
+
+    @Override
+    public PDone expand(PCollection<SendMessageRequest> input) {
+      input.apply(
+          ParDo.of(
+              new SqsWriteFn(
+                  new 
SqsConfiguration(input.getPipeline().getOptions().as(AwsOptions.class)))));
+      return PDone.in(input.getPipeline());
+    }
+  }
+
+  private static class SqsWriteFn extends DoFn<SendMessageRequest, Void> {
+    private final SqsConfiguration sqsConfiguration;
+    private transient AmazonSQS sqs;
+
+    SqsWriteFn(SqsConfiguration sqsConfiguration) {
+      this.sqsConfiguration = sqsConfiguration;
+    }
+
+    @Setup
+    public void setup() {
+      sqs =
+          AmazonSQSClientBuilder.standard()
+              
.withClientConfiguration(sqsConfiguration.getClientConfiguration())
+              .withCredentials(sqsConfiguration.getAwsCredentialsProvider())
+              .withRegion(sqsConfiguration.getAwsRegion())
+              .build();
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext processContext) throws Exception 
{
+      sqs.sendMessage(processContext.element());
+    }
+
+    @Teardown
+    public void teardown() throws Exception {
+      if (sqs != null) {
+        sqs.shutdown();
+      }
+    }
+  }
+}
diff --git 
a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java
 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java
new file mode 100644
index 00000000000..24bef192b3e
--- /dev/null
+++ 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java
@@ -0,0 +1,166 @@
+/*
+ * 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.aws.sqs;
+
+import com.amazonaws.services.sqs.model.Message;
+import com.amazonaws.services.sqs.model.MessageSystemAttributeName;
+import com.amazonaws.services.sqs.model.ReceiveMessageRequest;
+import com.amazonaws.services.sqs.model.ReceiveMessageResult;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.joda.time.Instant;
+
+class SqsUnboundedReader extends UnboundedSource.UnboundedReader<Message> 
implements Serializable {
+
+  public static final int MAX_NUMBER_OF_MESSAGES = 10;
+  private final SqsUnboundedSource source;
+  private Message current;
+  private final Queue<Message> messagesNotYetRead;
+  private List<Message> messagesToDelete;
+  private Instant oldestPendingTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+  public SqsUnboundedReader(SqsUnboundedSource source, SqsCheckpointMark 
sqsCheckpointMark) {
+    this.source = source;
+    this.current = null;
+
+    this.messagesNotYetRead = new ArrayDeque<>();
+    this.messagesToDelete = new ArrayList<>();
+
+    if (sqsCheckpointMark != null) {
+      this.messagesToDelete.addAll(sqsCheckpointMark.getMessagesToDelete());
+    }
+  }
+
+  @Override
+  public Instant getWatermark() {
+    return oldestPendingTimestamp;
+  }
+
+  @Override
+  public Message getCurrent() throws NoSuchElementException {
+    if (current == null) {
+      throw new NoSuchElementException();
+    }
+    return current;
+  }
+
+  @Override
+  public Instant getCurrentTimestamp() throws NoSuchElementException {
+    if (current == null) {
+      throw new NoSuchElementException();
+    }
+
+    return getTimestamp(current);
+  }
+
+  @Override
+  public byte[] getCurrentRecordId() throws NoSuchElementException {
+    if (current == null) {
+      throw new NoSuchElementException();
+    }
+    return current.getMessageId().getBytes(StandardCharsets.UTF_8);
+  }
+
+  @Override
+  public CheckpointMark getCheckpointMark() {
+    return new SqsCheckpointMark(this, messagesToDelete);
+  }
+
+  @Override
+  public SqsUnboundedSource getCurrentSource() {
+    return source;
+  }
+
+  @Override
+  public boolean start() {
+    return advance();
+  }
+
+  @Override
+  public boolean advance() {
+    if (messagesNotYetRead.isEmpty()) {
+      pull();
+    }
+
+    current = messagesNotYetRead.poll();
+    if (current == null) {
+      return false;
+    }
+
+    messagesToDelete.add(current);
+
+    Instant currentMessageTimestamp = getCurrentTimestamp();
+    if (getCurrentTimestamp().isBefore(oldestPendingTimestamp)) {
+      oldestPendingTimestamp = currentMessageTimestamp;
+    }
+
+    return true;
+  }
+
+  @Override
+  public void close() {}
+
+  void delete(final Collection<Message> messages) {
+    for (Message message : messages) {
+      if (messagesToDelete.contains(message)) {
+        source.getSqs().deleteMessage(source.getRead().queueUrl(), 
message.getReceiptHandle());
+        Instant currentMessageTimestamp = getTimestamp(message);
+        if (currentMessageTimestamp.isAfter(oldestPendingTimestamp)) {
+          oldestPendingTimestamp = currentMessageTimestamp;
+        }
+      }
+    }
+  }
+
+  private void pull() {
+    final ReceiveMessageRequest receiveMessageRequest =
+        new ReceiveMessageRequest(source.getRead().queueUrl());
+
+    receiveMessageRequest.setMaxNumberOfMessages(MAX_NUMBER_OF_MESSAGES);
+    receiveMessageRequest.setAttributeNames(
+        Arrays.asList(MessageSystemAttributeName.SentTimestamp.toString()));
+    final ReceiveMessageResult receiveMessageResult =
+        source.getSqs().receiveMessage(receiveMessageRequest);
+
+    final List<Message> messages = receiveMessageResult.getMessages();
+
+    if (messages == null || messages.size() == 0) {
+      return;
+    }
+
+    for (Message message : messages) {
+      messagesNotYetRead.add(message);
+    }
+  }
+
+  private Instant getTimestamp(final Message message) {
+    return new Instant(
+        Long.parseLong(
+            
message.getAttributes().get(MessageSystemAttributeName.SentTimestamp.toString())));
+  }
+}
diff --git 
a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedSource.java
 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedSource.java
new file mode 100644
index 00000000000..c79c5cc19c2
--- /dev/null
+++ 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedSource.java
@@ -0,0 +1,93 @@
+/*
+ * 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.aws.sqs;
+
+import com.amazonaws.services.sqs.AmazonSQS;
+import com.amazonaws.services.sqs.AmazonSQSClientBuilder;
+import com.amazonaws.services.sqs.model.Message;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import javax.annotation.Nullable;
+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.io.aws.sqs.SqsIO.Read;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+class SqsUnboundedSource extends UnboundedSource<Message, SqsCheckpointMark> {
+
+  private final Read read;
+  private final SqsConfiguration sqsConfiguration;
+  private final Supplier<AmazonSQS> sqs;
+
+  public SqsUnboundedSource(Read read, SqsConfiguration sqsConfiguration) {
+    this.read = read;
+    this.sqsConfiguration = sqsConfiguration;
+
+    sqs =
+        Suppliers.memoize(
+            (Supplier<AmazonSQS> & Serializable)
+                () ->
+                    AmazonSQSClientBuilder.standard()
+                        
.withClientConfiguration(sqsConfiguration.getClientConfiguration())
+                        
.withCredentials(sqsConfiguration.getAwsCredentialsProvider())
+                        .withRegion(sqsConfiguration.getAwsRegion())
+                        .build());
+  }
+
+  @Override
+  public List<SqsUnboundedSource> split(int desiredNumSplits, PipelineOptions 
options) {
+    List<SqsUnboundedSource> sources = new ArrayList<>();
+    for (int i = 0; i < Math.max(1, desiredNumSplits); ++i) {
+      sources.add(new SqsUnboundedSource(read, sqsConfiguration));
+    }
+    return sources;
+  }
+
+  @Override
+  public UnboundedReader<Message> createReader(
+      PipelineOptions options, @Nullable SqsCheckpointMark checkpointMark) {
+    return new SqsUnboundedReader(this, checkpointMark);
+  }
+
+  @Override
+  public Coder<SqsCheckpointMark> getCheckpointMarkCoder() {
+    return SerializableCoder.of(SqsCheckpointMark.class);
+  }
+
+  @Override
+  public Coder<Message> getOutputCoder() {
+    return SerializableCoder.of(Message.class);
+  }
+
+  public Read getRead() {
+    return read;
+  }
+
+  public AmazonSQS getSqs() {
+    return sqs.get();
+  }
+
+  @Override
+  public boolean requiresDeduping() {
+    return true;
+  }
+}
diff --git 
a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/package-info.java
 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/package-info.java
new file mode 100644
index 00000000000..d688641ddff
--- /dev/null
+++ 
b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/package-info.java
@@ -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.
+ */
+/** Defines IO connectors for Amazon Web Services SQS. */
+package org.apache.beam.sdk.io.aws.sqs;
diff --git 
a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sqs/SqsIOTest.java
 
b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sqs/SqsIOTest.java
new file mode 100644
index 00000000000..327cc9e1d5a
--- /dev/null
+++ 
b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sqs/SqsIOTest.java
@@ -0,0 +1,139 @@
+/*
+ * 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.aws.sqs;
+
+import static org.junit.Assert.assertEquals;
+
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.sqs.AmazonSQS;
+import com.amazonaws.services.sqs.AmazonSQSClientBuilder;
+import com.amazonaws.services.sqs.model.CreateQueueResult;
+import com.amazonaws.services.sqs.model.Message;
+import com.amazonaws.services.sqs.model.ReceiveMessageResult;
+import com.amazonaws.services.sqs.model.SendMessageRequest;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.elasticmq.rest.sqs.SQSRestServer;
+import org.elasticmq.rest.sqs.SQSRestServerBuilder;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExternalResource;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests on {@link SqsIO}. */
+@RunWith(JUnit4.class)
+public class SqsIOTest {
+
+  @Rule public TestPipeline pipeline = TestPipeline.create();
+
+  @Rule public EmbeddedSqsServer embeddedSqsRestServer = new 
EmbeddedSqsServer();
+
+  @Test
+  public void testRead() {
+    final AmazonSQS client = embeddedSqsRestServer.getClient();
+    final String queueUrl = embeddedSqsRestServer.getQueueUrl();
+
+    final PCollection<Message> output =
+        
pipeline.apply(SqsIO.read().withQueueUrl(queueUrl).withMaxNumRecords(100));
+
+    PAssert.thatSingleton(output.apply(Count.globally())).isEqualTo(100L);
+
+    for (int i = 0; i < 100; i++) {
+      client.sendMessage(queueUrl, "This is a test");
+    }
+    pipeline.run();
+  }
+
+  @Test
+  public void testWrite() {
+    final AmazonSQS client = embeddedSqsRestServer.getClient();
+    final String queueUrl = embeddedSqsRestServer.getQueueUrl();
+
+    List<SendMessageRequest> messages = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      final SendMessageRequest request = new SendMessageRequest(queueUrl, 
"This is a test " + i);
+      messages.add(request);
+    }
+    pipeline.apply(Create.of(messages)).apply(SqsIO.write());
+    pipeline.run().waitUntilFinish();
+
+    List<String> received = new ArrayList<>();
+    while (received.size() < 100) {
+      final ReceiveMessageResult receiveMessageResult = 
client.receiveMessage(queueUrl);
+
+      if (receiveMessageResult.getMessages() != null) {
+        for (Message message : receiveMessageResult.getMessages()) {
+          received.add(message.getBody());
+        }
+      }
+    }
+    assertEquals(100, received.size());
+    for (int i = 0; i < 100; i++) {
+      received.contains("This is a test " + i);
+    }
+  }
+
+  private static class EmbeddedSqsServer extends ExternalResource {
+
+    private SQSRestServer sqsRestServer;
+    private AmazonSQS client;
+    private String queueUrl;
+
+    @Override
+    protected void before() {
+      sqsRestServer = SQSRestServerBuilder.start();
+
+      String endpoint = "http://localhost:9324";;
+      String region = "elasticmq";
+      String accessKey = "x";
+      String secretKey = "x";
+
+      client =
+          AmazonSQSClientBuilder.standard()
+              .withCredentials(
+                  new AWSStaticCredentialsProvider(new 
BasicAWSCredentials(accessKey, secretKey)))
+              .withEndpointConfiguration(
+                  new AwsClientBuilder.EndpointConfiguration(endpoint, region))
+              .withRegion(region)
+              .build();
+      final CreateQueueResult queue = client.createQueue("test");
+      queueUrl = queue.getQueueUrl();
+    }
+
+    @Override
+    protected void after() {
+      sqsRestServer.stopAndWait();
+    }
+
+    public AmazonSQS getClient() {
+      return client;
+    }
+
+    public String getQueueUrl() {
+      return queueUrl;
+    }
+  }
+}


 

----------------------------------------------------------------
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: 137137)
    Time Spent: 12h  (was: 11h 50m)

> Add Amazon SqsIO
> ----------------
>
>                 Key: BEAM-4828
>                 URL: https://issues.apache.org/jira/browse/BEAM-4828
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-aws
>            Reporter: John Rudolf Lewis
>            Assignee: John Rudolf Lewis
>            Priority: Major
>          Time Spent: 12h
>  Remaining Estimate: 0h
>
> Add an SQS source
>  
> For people who would like to follow progress or help out: 
> [https://github.com/JohnRudolfLewis/beam/tree/Add-SqsIO]



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

Reply via email to