I’m ok either having something generic that calls the FluentBackoff internally or making FluentBackoff available for public. Whichever way the community decides I can change all IO’s to comply to the standard
From: Alexey Romanenko <[email protected]> Reply-To: "[email protected]" <[email protected]> Date: Tuesday, April 21, 2020 at 9:42 AM To: "[email protected]" <[email protected]> Subject: Re: ** Configurable FluentBackoff for IO's ** Notice: This email is from an external sender. I can guess it was done in this way to avoid using a class from “org.apache.beam.sdk.util” package. Can we just move FluentBackoff from “org.apache.beam.sdk.util” package to another package, available for users, or to create a common wrapper for such cases, like IO retries? On 17 Apr 2020, at 21:14, Luke Cwik <[email protected]<mailto:[email protected]>> wrote: On Fri, Apr 17, 2020 at 9:57 AM Alexey Romanenko <[email protected]<mailto:[email protected]>> wrote: As we can see, that support of Backoff in some way is quite demanded feature for different IOs. Of course, we don’t want to expose too many knobs but seems that this “backoff knob" should be able to be configured by user since it depends on different aspects of its environment. In the PR mentioned by Jonothan, we discussed that FluentBackoff was not exposed since it’s a part of “org.apache.beam.sdk.util” package which is for internal use only. Since many IOs already use this by wrapping it around own API classes, why not to make this FluentBackoff as a part of public API? That is what we are trying to answer. Why did those implementations decide to wrap it instead of exposing it. On 17 Apr 2020, at 17:16, Luke Cwik <[email protected]<mailto:[email protected]>> wrote: Jonothan, you're still on point because exposing and/or using the client specific retry implementation is a valid strategy as it exposes all the knobs that a user may want to use. A downside I can see is that it may expose knobs that are irrelevant for the transform or makes it difficult to integrate other forms of retry that are specific to the transform outside of what the client library may do such as what to do with failed records being processed (retried, goto a DLQ, be dropped). Looking through the code for more examples, I see everyone rolling their own instead of exposing FluentBackoff or exposing client specific retry implementations: DynamoDBIO: https://github.com/apache/beam/blob/a1b79fdc995c869d1f32fab2e2004621b2d53988/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIO.java#L290 ElasticSearchIO: https://github.com/apache/beam/blob/a1b79fdc995c869d1f32fab2e2004621b2d53988/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java#L937 ClickHouseIO: https://github.com/apache/beam/blob/a1b79fdc995c869d1f32fab2e2004621b2d53988/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java#L258 On Fri, Apr 17, 2020 at 8:14 AM Chamikara Jayalath <[email protected]<mailto:[email protected]>> wrote: Another option might be to add explicitly defined retry policies to the API. For example, see following for BigQueryIO. https://github.com/apache/beam/blob/a1b79fdc995c869d1f32fab2e2004621b2d53988/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java On Thu, Apr 16, 2020 at 9:48 PM Akshay Iyangar <[email protected]<mailto:[email protected]>> wrote: Luke I think for [2] and [3] it would be a fair statement that may be they wanted to add a custom retry configuration. But [2] looks very specific in the sense it doesn’t allow client to be more flexible [3] is something that I feel can be moved up and made generic enough. Jonothan Sorry for that, this was actually with regards to JdbcIO. My bad calling it S3. From: Jonothan Farr <[email protected]<mailto:[email protected]>> Reply-To: "[email protected]<mailto:[email protected]>" <[email protected]<mailto:[email protected]>> Date: Thursday, April 16, 2020 at 7:07 PM To: "[email protected]<mailto:[email protected]>" <[email protected]<mailto:[email protected]>> Subject: Re: ** Configurable FluentBackoff for IO's ** Notice: This email is from an external sender. Maybe this is a separate conversation, but for AWS IOs specifically wouldn't it be better to use the AWS client's retry policy? Something similar to this: ``` @Override public AmazonS3ClientBuilder createBuilder(S3Options s3Options) { RetryPolicy retryPolicy = new RetryPolicy( PredefinedRetryPolicies.DEFAULT_RETRY_CONDITION, PredefinedRetryPolicies.DEFAULT_BACKOFF_STRATEGY, PredefinedRetryPolicies.DEFAULT_MAX_ERROR_RETRY, false); AmazonS3ClientBuilder builder = AmazonS3ClientBuilder.standard() .withClientConfiguration(PredefinedClientConfigurations.defaultConfig() .withRetryPolicy(retryPolicy)) .withCredentials(s3Options.getAwsCredentialsProvider()); ... ``` We had a similar discussion on https://github.com/apache/beam/pull/9765 about KinesisIO. I only bring it up because you mentioned configuring retries in S3. On Thu, Apr 16, 2020 at 1:57 PM Luke Cwik <[email protected]<mailto:[email protected]>> wrote: I was wondering why IOs went with their own retry configuration object instead of making FluentBackoff[1] public. Some examples are SnsIO[2] and SolrIO[3]. Was it because we thought that IOs would likely need specialized retry configuration that a general retry configuration class wouldn't apply? 1: https://github.com/apache/beam/blob/c3bd4854e879da65060de8cd259865a9b34742c7/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java#L30 2: https://github.com/apache/beam/blob/da9e17288e8473925674a4691d9e86252e67d7d7/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java#L262 3: https://github.com/apache/beam/blob/da9e17288e8473925674a4691d9e86252e67d7d7/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java#L225 On Wed, Apr 15, 2020 at 11:59 AM Akshay Iyangar <[email protected]<mailto:[email protected]>> wrote: Hi I actually wanted a way to configure FluentBackoff at the client side for S3 in that effort I created below PR. But as luke mentioned in the PR FluentBackoff is part of util and I can directly expose it to public. So a suggested alternative was to use a Configuration class that is public facing which then convert’s it to the internal beam class and have it generic enough to be used across IO’s. Just wanted to know what the community feels and if the above suggestion by luke is ok with other’s I’ll try to implement that instead. JIRA - 9742 https://github.com/apache/beam/pull/11396 Thanks Akshay I
