[ 
https://issues.apache.org/jira/browse/BEAM-10673?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dennis Yung updated BEAM-10673:
-------------------------------
    Description: 
As described in the documentation, DynamoDBIO is initiated with a 
RetryConfiguration through withRetryConfiguration()
{code:java}
 PCollection<T> data = ...;
 data.apply(
           DynamoDBIO.<WriteRequest>write()
               .withWriteRequestMapperFn(
                   (SerializableFunction<T, KV<String, WriteRequest>>)
                       //Transforming your T data into KV<String, WriteRequest>
                       t -> KV.of(tableName, writeRequest))
               .withRetryConfiguration(
                    DynamoDBIO.RetryConfiguration.create(5, 
Duration.standardMinutes(1)))
               .withAwsClientsProvider(new BasicDynamoDbProvider(accessKey, 
secretKey, region));
{code}
It works on beam-sdks-java-io-amazon-web-services (aws sdk version 1), but not 
on beam-sdks-java-io-amazon-web-services2 (aws sdk version 2).

It is because io.aws2.dynamodb.DynamoDBIO.RetryConfiguration adopted a 
AutoValue builder design in place of the create method. However, the builder do 
not have public methods for setting values and building a RetryConfiguration. 
Moreover, validation was not implemented
{code:java}
  @AutoValue
  public abstract static class RetryConfiguration implements Serializable {
    
   /.../

    public static Builder builder() {
      return new AutoValue_DynamoDBIO_RetryConfiguration.Builder();
    }

    @AutoValue.Builder
    abstract static class Builder {
      abstract Builder setMaxAttempts(int maxAttempts);

      abstract Builder setMaxDuration(Duration maxDuration);

      abstract Builder setRetryPredicate(RetryPredicate retryPredicate);

      abstract RetryConfiguration build();
    }
{code}

  was:
As described in the documentation, DynamoDBIO is initiated with a 
RetryConfiguration passed in withRetryConfiguration()
{code:java}
 PCollection<T> data = ...;
 data.apply(
           DynamoDBIO.<WriteRequest>write()
               .withWriteRequestMapperFn(
                   (SerializableFunction<T, KV<String, WriteRequest>>)
                       //Transforming your T data into KV<String, WriteRequest>
                       t -> KV.of(tableName, writeRequest))
               .withRetryConfiguration(
                    DynamoDBIO.RetryConfiguration.create(5, 
Duration.standardMinutes(1)))
               .withAwsClientsProvider(new BasicDynamoDbProvider(accessKey, 
secretKey, region));
{code}
It works with beam-sdks-java-io-amazon-web-services (aws sdk version 1), but 
not with beam-sdks-java-io-amazon-web-services2 (aws sdk version 2).

It is because in io.aws2.dynamodb.DynamoDBIO.RetryConfiguration, the method 
create() is missing. There does not seem to be alternative public constructors 
either.


> DynamoDBIO.RetryConfiguration does not have public constructor
> --------------------------------------------------------------
>
>                 Key: BEAM-10673
>                 URL: https://issues.apache.org/jira/browse/BEAM-10673
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-aws
>    Affects Versions: 2.23.0
>            Reporter: Dennis Yung
>            Priority: P2
>
> As described in the documentation, DynamoDBIO is initiated with a 
> RetryConfiguration through withRetryConfiguration()
> {code:java}
>  PCollection<T> data = ...;
>  data.apply(
>            DynamoDBIO.<WriteRequest>write()
>                .withWriteRequestMapperFn(
>                    (SerializableFunction<T, KV<String, WriteRequest>>)
>                        //Transforming your T data into KV<String, 
> WriteRequest>
>                        t -> KV.of(tableName, writeRequest))
>                .withRetryConfiguration(
>                     DynamoDBIO.RetryConfiguration.create(5, 
> Duration.standardMinutes(1)))
>                .withAwsClientsProvider(new BasicDynamoDbProvider(accessKey, 
> secretKey, region));
> {code}
> It works on beam-sdks-java-io-amazon-web-services (aws sdk version 1), but 
> not on beam-sdks-java-io-amazon-web-services2 (aws sdk version 2).
> It is because io.aws2.dynamodb.DynamoDBIO.RetryConfiguration adopted a 
> AutoValue builder design in place of the create method. However, the builder 
> do not have public methods for setting values and building a 
> RetryConfiguration. Moreover, validation was not implemented
> {code:java}
>   @AutoValue
>   public abstract static class RetryConfiguration implements Serializable {
>     
>    /.../
>     public static Builder builder() {
>       return new AutoValue_DynamoDBIO_RetryConfiguration.Builder();
>     }
>     @AutoValue.Builder
>     abstract static class Builder {
>       abstract Builder setMaxAttempts(int maxAttempts);
>       abstract Builder setMaxDuration(Duration maxDuration);
>       abstract Builder setRetryPredicate(RetryPredicate retryPredicate);
>       abstract RetryConfiguration build();
>     }
> {code}



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

Reply via email to