You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2019/11/18 17:55:00 UTC

[jira] [Work logged] (BEAM-7636) Migrate SqsIO to AWS SDK for Java 2

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

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

                Author: ASF GitHub Bot
            Created on: 18/Nov/19 17:54
            Start Date: 18/Nov/19 17:54
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on pull request #9935: [BEAM-7636] Migrate SqsIO to AWS SDK for Java 2
URL: https://github.com/apache/beam/pull/9935#discussion_r347514151
 
 

 ##########
 File path: sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java
 ##########
 @@ -0,0 +1,272 @@
+/*
+ * 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.aws2.sqs;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import java.net.URI;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.aws2.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;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.services.sqs.SqsClient;
+import software.amazon.awssdk.services.sqs.model.SendMessageRequest;
+
+/**
+ * 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
+ * software.amazon.awssdk.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<SqsMessage>> {
+
+    @Nullable
+    abstract String queueUrl();
+
+    abstract long maxNumRecords();
+
+    @Nullable
+    abstract Duration maxReadTime();
+
+    @Nullable
+    abstract SqsClientProvider sqsClientProvider();
+
+    abstract Builder builder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setQueueUrl(String queueUrl);
+
+      abstract Builder setMaxNumRecords(long maxNumRecords);
+
+      abstract Builder setMaxReadTime(Duration maxReadTime);
+
+      abstract Builder setSqsClientProvider(SqsClientProvider sqsClientProvider);
+
+      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 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) {
+      return builder().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 builder().setQueueUrl(queueUrl).build();
+    }
+
+    /**
+     * Allows to specify custom {@link SqsClientProvider}. {@link SqsClientProvider} creates new
+     * {@link SqsClient} which is later used for writing to a SqS queue.
+     */
+    public Read withSqsClientProvider(SqsClientProvider awsClientsProvider) {
+      return builder().setSqsClientProvider(awsClientsProvider).build();
+    }
+
+    /**
+     * Specify {@link software.amazon.awssdk.auth.credentials.AwsCredentialsProvider} and region to
+     * be used to read from SQS. If you need more sophisticated credential protocol, then you should
+     * look at {@link Read#withSqsClientProvider(SqsClientProvider)}.
+     */
+    public Read withSqsClientProvider(AwsCredentialsProvider credentialsProvider, String region) {
+      return withSqsClientProvider(credentialsProvider, region, null);
+    }
+
+    /**
+     * Specify {@link AwsCredentialsProvider} and region to be used to write to SQS. If you need
+     * more sophisticated credential protocol, then you should look at {@link
+     * Read#withSqsClientProvider(SqsClientProvider)}.
+     *
+     * <p>The {@code serviceEndpoint} sets an alternative service host. This is useful to execute
+     * the tests with Kinesis service emulator.
+     */
+    public Read withSqsClientProvider(
+        AwsCredentialsProvider credentialsProvider, String region, URI serviceEndpoint) {
+      return withSqsClientProvider(
+          new BasicSqsClientProvider(credentialsProvider, region, serviceEndpoint));
+    }
+
+    @Override
+    public PCollection<SqsMessage> expand(PBegin input) {
+
+      org.apache.beam.sdk.io.Read.Unbounded<SqsMessage> unbounded =
+          org.apache.beam.sdk.io.Read.from(new SqsUnboundedSource(this));
+
+      PTransform<PBegin, PCollection<SqsMessage>> 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> {
+
+    @Nullable
+    abstract SqsClientProvider getSqsClientProvider();
+
+    abstract Builder builder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+
+      abstract Builder setSqsClientProvider(SqsClientProvider sqsClientProvider);
+
+      abstract Write build();
+    }
+
+    /**
+     * Allows to specify custom {@link SqsClientProvider}. {@link SqsClientProvider} creates new
+     * {@link SqsClient} which is later used for writing to a SqS queue.
+     */
+    public Write withSqsClientProvider(SqsClientProvider awsClientsProvider) {
+      return builder().setSqsClientProvider(awsClientsProvider).build();
+    }
+
+    /**
+     * Specify {@link software.amazon.awssdk.auth.credentials.AwsCredentialsProvider} and region to
+     * be used to write to SQS. If you need more sophisticated credential protocol, then you should
+     * look at {@link Write#withSqsClientProvider(SqsClientProvider)}.
+     */
+    public Write withSqsClientProvider(AwsCredentialsProvider credentialsProvider, String region) {
+      return withSqsClientProvider(credentialsProvider, region, null);
+    }
+
+    /**
+     * Specify {@link AwsCredentialsProvider} and region to be used to write to SQS. If you need
+     * more sophisticated credential protocol, then you should look at {@link
+     * Write#withSqsClientProvider(SqsClientProvider)}.
+     *
+     * <p>The {@code serviceEndpoint} sets an alternative service host. This is useful to execute
+     * the tests with Kinesis service emulator.
+     */
+    public Write withSqsClientProvider(
+        AwsCredentialsProvider credentialsProvider, String region, URI serviceEndpoint) {
+      return withSqsClientProvider(
+          new BasicSqsClientProvider(credentialsProvider, region, serviceEndpoint));
+    }
+
+    @Override
+    public PDone expand(PCollection<SendMessageRequest> input) {
+      input.apply(ParDo.of(new SqsWriteFn(this)));
+      return PDone.in(input.getPipeline());
+    }
+  }
+
+  private static class SqsWriteFn extends DoFn<SendMessageRequest, Void> {
+    private final Write spec;
+    private transient SqsClient sqs;
+
+    SqsWriteFn(Write write) {
+      this.spec = write;
+    }
+
+    @Setup
+    public void setup() {
+      sqs = spec.getSqsClientProvider().getSqsClient();
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext processContext) throws Exception {
+      sqs.sendMessage(processContext.element());
 
 Review comment:
   Does it send message synchronically? Would it make sense to batch the messages of the whole bundle and send them as a batch for better performance? 
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 345465)
    Time Spent: 0.5h  (was: 20m)

> Migrate SqsIO to AWS SDK for Java 2
> -----------------------------------
>
>                 Key: BEAM-7636
>                 URL: https://issues.apache.org/jira/browse/BEAM-7636
>             Project: Beam
>          Issue Type: Sub-task
>          Components: io-java-aws
>            Reporter: Ismaël Mejía
>            Assignee: Cam Mach
>            Priority: Major
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)