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 2021/11/05 15:51:00 UTC

[jira] [Work logged] (BEAM-13192) Tests for retries in SnsIO.Writer are buggy

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

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

                Author: ASF GitHub Bot
            Created on: 05/Nov/21 15:50
            Start Date: 05/Nov/21 15:50
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on a change in pull request #15910:
URL: https://github.com/apache/beam/pull/15910#discussion_r743764621



##########
File path: sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/SnsIOTest.java
##########
@@ -114,7 +118,10 @@ public void testDataWritesToSNS() {
 
   @Test
   public void testRetries() throws Throwable {
+    thrown.expect(IOException.class);

Review comment:
       Is it an exact exception that we expect?

##########
File path: sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java
##########
@@ -111,25 +111,39 @@ public static Write write() {
   @AutoValue.CopyAnnotations
   @SuppressWarnings({"rawtypes"})
   public abstract static class RetryConfiguration implements Serializable {
+    private static final Duration DEFAULT_INITIAL_DURATION = Duration.standardSeconds(5);
+
     @VisibleForTesting
     static final RetryPredicate DEFAULT_RETRY_PREDICATE = new DefaultRetryPredicate();
 
     abstract int getMaxAttempts();
 
     abstract Duration getMaxDuration();
 
+    abstract Duration getInitialDuration();
+
     abstract RetryPredicate getRetryPredicate();
 
     abstract Builder builder();
 
-    public static RetryConfiguration create(int maxAttempts, Duration maxDuration) {
+    @VisibleForTesting
+    static RetryConfiguration create(int maxAttempts, Duration maxDuration) {

Review comment:
       We should keep it public, otherwise it will be a breaking user API change. 

##########
File path: sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/SnsIOTest.java
##########
@@ -114,7 +118,10 @@ public void testDataWritesToSNS() {
 
   @Test
   public void testRetries() throws Throwable {
+    thrown.expect(IOException.class);
     thrown.expectMessage("Error writing to SNS");
+    thrown.expectMessage("No more attempts allowed");

Review comment:
       Does it expect two exception messages?

##########
File path: sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/SnsIOTest.java
##########
@@ -123,20 +130,19 @@ public void testRetries() throws Throwable {
             SnsIO.write()
                 .withTopicName(topicName)
                 .withRetryConfiguration(
-                    SnsIO.RetryConfiguration.create(4, org.joda.time.Duration.standardSeconds(10)))
+                    SnsIO.RetryConfiguration.create(4, standardSeconds(10), millis(1)))
                 .withAWSClientsProvider(new Provider(amazonSnsErrors))
                 .withResultOutputTag(results));
 
     try {
       p.run();
     } catch (final Pipeline.PipelineExecutionException e) {
       // check 3 retries were initiated by inspecting the log before passing on the exception
-      expectedLogs.verifyWarn(String.format(SnsIO.Write.SnsWriterFn.RETRY_ATTEMPT_LOG, 1));
-      expectedLogs.verifyWarn(String.format(SnsIO.Write.SnsWriterFn.RETRY_ATTEMPT_LOG, 2));
-      expectedLogs.verifyWarn(String.format(SnsIO.Write.SnsWriterFn.RETRY_ATTEMPT_LOG, 3));
+      snsWriterFnLogs.verifyWarn(String.format(SnsIO.Write.SnsWriterFn.RETRY_ATTEMPT_LOG, 1));
+      snsWriterFnLogs.verifyWarn(String.format(SnsIO.Write.SnsWriterFn.RETRY_ATTEMPT_LOG, 2));
+      snsWriterFnLogs.verifyWarn(String.format(SnsIO.Write.SnsWriterFn.RETRY_ATTEMPT_LOG, 3));
       throw e.getCause();
     }
-    fail("Pipeline is expected to fail because we were unable to write to SNS.");

Review comment:
       Why it's not needed anymore?




-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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

            Worklog Id:     (was: 677070)
    Remaining Estimate: 0h
            Time Spent: 10m

> Tests for retries in SnsIO.Writer are buggy
> -------------------------------------------
>
>                 Key: BEAM-13192
>                 URL: https://issues.apache.org/jira/browse/BEAM-13192
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-aws
>            Reporter: Moritz Mack
>            Assignee: Moritz Mack
>            Priority: P3
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> There's no working verification that retries happened. 
> All assertions fail in unforeseen / unintended ways.



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