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

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

                Author: ASF GitHub Bot
            Created on: 29/Apr/20 16:45
            Start Date: 29/Apr/20 16:45
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on a change in pull request 
#10078:
URL: https://github.com/apache/beam/pull/10078#discussion_r417371772



##########
File path: 
sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/BasicSnsAsyncClientProvider.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.sns;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.net.URI;
+import javax.annotation.Nullable;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.sns.SnsAsyncClient;
+import software.amazon.awssdk.services.sns.SnsAsyncClientBuilder;
+
+/** Basic implementation of {@link SnsClientProvider} used by default in 
{@link SnsIO}. */

Review comment:
       `Basic implementation of {@ SnsAsyncClientProvider}` maybe ?

##########
File path: 
sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOWriteTest.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.sns;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Set;
+import java.util.stream.StreamSupport;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import software.amazon.awssdk.services.sns.model.PublishRequest;
+
+@RunWith(JUnit4.class)
+public class SnsIOWriteTest implements Serializable {
+  private static final String TOPIC = "test";
+  private static final int FAILURE_STATUS_CODE = 400;
+  private static final int SUCCESS_STATUS_CODE = 200;
+
+  @Rule public transient TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void shouldReturnResponseOnPublishSuccess() {
+    String testMessage1 = "test1";
+    String testMessage2 = "test2";
+    String testMessage3 = "test3";
+
+    PCollection<SnsResponse<String>> result =
+        pipeline
+            .apply(
+                Create.of(testMessage1, testMessage2, 
testMessage3).withCoder(StringUtf8Coder.of()))
+            .apply(
+                SnsIO.<String>asyncWrite()
+                    .withCoder(StringUtf8Coder.of())
+                    .withPublishRequestFn(createPublishRequestFn())
+                    .withSnsClientProvider(
+                        () -> 
MockSnsAsyncClient.withStatusCode(SUCCESS_STATUS_CODE)));
+
+    PAssert.that(result)
+        .satisfies(
+            (responses) -> {
+              ImmutableSet<String> messagesInResponse =
+                  StreamSupport.stream(responses.spliterator(), false)
+                      .filter(response -> response.statusCode().getAsInt() == 
SUCCESS_STATUS_CODE)
+                      .map(SnsResponse::element)
+                      .collect(ImmutableSet.toImmutableSet());
+
+              Set<String> originalMessages =
+                  Sets.newHashSet(testMessage1, testMessage2, testMessage3);
+              Sets.SetView<String> difference =
+                  Sets.difference(messagesInResponse, originalMessages);
+
+              assertEquals(3, messagesInResponse.size());
+              assertEquals(0, difference.size());
+              return null;
+            });
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void shouldReturnResponseOnPublishFailure() {
+    String testMessage1 = "test1";
+    String testMessage2 = "test2";
+
+    PCollection<SnsResponse<String>> result =
+        pipeline
+            .apply(Create.of(testMessage1, 
testMessage2).withCoder(StringUtf8Coder.of()))
+            .apply(
+                SnsIO.<String>asyncWrite()
+                    .withCoder(StringUtf8Coder.of())
+                    .withPublishRequestFn(createPublishRequestFn())
+                    .withSnsClientProvider(
+                        () -> 
MockSnsAsyncClient.withStatusCode(FAILURE_STATUS_CODE)));
+
+    PAssert.that(result)
+        .satisfies(
+            (responses) -> {
+              ImmutableSet<String> messagesInResponse =
+                  StreamSupport.stream(responses.spliterator(), false)
+                      .filter(response -> response.statusCode().getAsInt() != 
SUCCESS_STATUS_CODE)
+                      .map(SnsResponse::element)
+                      .collect(ImmutableSet.toImmutableSet());
+
+              Set<String> originalMessages = Sets.newHashSet(testMessage1, 
testMessage2);
+              Sets.SetView<String> difference =
+                  Sets.difference(messagesInResponse, originalMessages);
+
+              assertEquals(2, messagesInResponse.size());
+              assertEquals(0, difference.size());
+              return null;
+            });
+    pipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  @SuppressWarnings("MissingFail")
+  public void shouldThrowIfThrowErrorOptionSet() {
+    String testMessage1 = "test1";
+
+    pipeline
+        .apply(Create.of(testMessage1))
+        .apply(
+            SnsIO.<String>asyncWrite()
+                .withCoder(StringUtf8Coder.of())
+                .withPublishRequestFn(createPublishRequestFn())
+                .withSnsClientProvider(
+                    () -> 
MockSnsAsyncClient.withStatusCode(FAILURE_STATUS_CODE)));
+    try {
+      pipeline.run().waitUntilFinish();
+    } catch (final Pipeline.PipelineExecutionException e) {
+      Assert.assertEquals(IOException.class, e.getCause().getClass());

Review comment:
       Perhaps, better to use `Assert.assertThrows()` for this

##########
File path: 
sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsResponseCoder.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.sns;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalInt;
+import org.apache.beam.sdk.coders.BooleanCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.StructuredCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+/** Custom Coder for WrappedSnsResponse. */
+class SnsResponseCoder<T> extends StructuredCoder<SnsResponse<T>> {

Review comment:
       Please, add tests for this class

##########
File path: 
sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java
##########
@@ -79,15 +83,51 @@
  *   <li>AwsCredentialsProvider, which you can pass on to 
BasicSnsClientProvider
  *   <li>publishRequestFn, a function to convert your message into 
PublishRequest
  * </ul>
+ *
+ * <h3>Writing to SNS Asynchronously</h3>
+ *
+ * <p>Example usage:
+ *
+ * <pre>{@code
+ * PCollection<String> data = ...;
+ *
+ * data.apply(SnsIO.<String>asyncWrite()
+ *             .withElementCoder(StringUtf8Coder.of())
+ *             .withPublishRequestFn(createPublishRequestFn())
+ *             .withSnsClientProvider(new 
BasicSnsClientProvider(awsCredentialsProvider, region));
+ *
+ * }</pre>
+ *
+ * <pre>{@code
+ * PCollection<String> data = ...;
+ *
+ * PCollection<SnsResponse<String>> responses = 
data.apply(SnsIO.<String>asyncWrite()
+ *      .withElementCoder(StringUtf8Coder.of())
+ *      .withPublishRequestFn(createPublishRequestFn())
+ *  *   .withSnsClientProvider(new 
BasicSnsClientProvider(awsCredentialsProvider, region));
+ *
+ * }</pre>
+ *
+ * <p>As a client, you need to provide at least the following things:
+ *
+ * <ul>
+ *   <li>Coder for element T.
+ *   <li>publishRequestFn, a function to convert your message into 
PublishRequest
+ *   <li>SnsClientProvider, a provider to create an async client.
+ * </ul>
  */
 @Experimental(Kind.SOURCE_SINK)
 public final class SnsIO {
 
-  // Write data tp SNS
+  // Write data to SNS (synchronous)
   public static <T> Write<T> write() {
     return new AutoValue_SnsIO_Write.Builder().build();
   }
 
+  public static <T> AsyncWrite<T> asyncWrite() {

Review comment:
       Please, rename it to `writeAsync()`, better to have a verb as a first 
word for methods.




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


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

    Worklog Id:     (was: 428658)
    Time Spent: 9h 10m  (was: 9h)

> Add async write to AWS SNS IO & remove retry logic
> --------------------------------------------------
>
>                 Key: BEAM-8542
>                 URL: https://issues.apache.org/jira/browse/BEAM-8542
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-aws
>            Reporter: Ajo Thomas
>            Assignee: Ajo Thomas
>            Priority: Major
>          Time Spent: 9h 10m
>  Remaining Estimate: 0h
>
> - While working with SNS IO for one of my work-related projects, I found that 
> the IO uses synchronous publishes during writes. I had a simple mock pipeline 
> where I was reading from a kinesis stream and publishing it to SNS using 
> Beam's SNS IO. For comparison, I also had a lamdba which did the same using 
> asynchronous publishes but was about 5x faster. Changing the SNS IO to use 
> async publishes would improve publish latencies.
>  - SNS IO also has some retry logic which isn't required as SNS clients can 
> handle retries. The retry logic in the SNS client is user-configurable and 
> therefore, an explicit retry logic in SNS IO is not required
> I have a working version of the IO with these changes, will create a PR 
> linking this ticket to it once I get some feedback here.



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

Reply via email to