[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2023-09-07 Thread Satish Duggana (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17762921#comment-17762921
 ] 

Satish Duggana commented on KAFKA-9800:
---

[~junrao]  [~schofielaj] Are you planning to merge these changes to 3.6 branch?

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Andrew Schofield
>Priority: Major
>  Labels: KIP-580, client
> Fix For: 3.7.0
>
>
> Design:
> The main idea is to bookkeep the failed attempt. Currently, the retry backoff 
> has two main usage patterns:
>  # Synchronous retires and blocking loop. The thread will sleep in each 
> iteration for retry backoff ms.
>  # Async retries. In each polling, the retries do not meet the backoff will 
> be filtered. The data class often maintains a 1:1 mapping to a set of 
> requests which are logically associated. (i.e. a set contains only one 
> initial request and only its retries.)
> For type 1, we can utilize a local failure counter of a Java generic data 
> type.
> For case 2, I already wrapped the exponential backoff/timeout util class in 
> my KIP-601 
> [implementation|https://github.com/apache/kafka/pull/8683/files#diff-9ca2b1294653dfa914b9277de62b52e3R28]
>  which takes the number of attempts and returns the backoff/timeout value at 
> the corresponding level. Thus, we can add a new class property to those 
> classes containing retriable data in order to record the number of failed 
> attempts.
>  
> Changes:
> KafkaProducer:
>  # Produce request (ApiKeys.PRODUCE). Currently, the backoff applies to each 
> ProducerBatch in Accumulator, which already has an attribute attempts 
> recording the number of failed attempts. So we can let the Accumulator 
> calculate the new retry backoff for each bach when it enqueues them, to avoid 
> instantiate the util class multiple times.
>  # Transaction request (ApiKeys..*TXN). TxnRequestHandler will have a new 
> class property of type `Long` to record the number of attempts.
> KafkaConsumer:
>  # Some synchronous retry use cases. Record the failed attempts in the 
> blocking loop.
>  # Partition request (ApiKeys.OFFSET_FOR_LEADER_EPOCH, ApiKeys.LIST_OFFSETS). 
> Though the actual requests are packed for each node, the current 
> implementation is applying backoff to each topic partition, where the backoff 
> value is kept by TopicPartitionState. Thus, TopicPartitionState will have the 
> new property recording the number of attempts.
> Metadata:
>  #  Metadata lives as a singleton in many clients. Add a new property 
> recording the number of attempts
>  AdminClient:
>  # AdminClient has its own request abstraction Call. The failed attempts are 
> already kept by the abstraction. So probably clean the Call class logic a bit.
> Existing tests:
>  # If the tests are testing the retry backoff, add a delta to the assertion, 
> considering the existence of the jitter.
>  # If the tests are testing other functionality, we can specify the same 
> value for both `retry.backoff.ms` and `retry.backoff.max.ms` in order to make 
> the retry backoff static. We can use this trick to make the existing tests 
> compatible with the changes.
> There're other common usages look like client.poll(timeout), where the 
> timeout passed in is the retry backoff value. We won't change these usages 
> since its underlying logic is nioSelector.select(timeout) and 
> nioSelector.selectNow(), which means if no interested op exists, the client 
> will block retry backoff milliseconds. This is an optimization when there's 
> no request that needs to be sent but the client is waiting for responses. 
> Specifically, if the client fails the inflight requests before the retry 
> backoff milliseconds passed, it still needs to wait until that amount of time 
> passed, unless there's a new request need to be sent.
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2023-06-22 Thread Andrew Schofield (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17736180#comment-17736180
 ] 

Andrew Schofield commented on KAFKA-9800:
-

[~showuon] I wonder what the current status of this is. I'd be interested in 
helping to get it delivered in a forthcoming release.

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Luke Chen
>Priority: Major
>  Labels: KIP-580, client
>
> Design:
> The main idea is to bookkeep the failed attempt. Currently, the retry backoff 
> has two main usage patterns:
>  # Synchronous retires and blocking loop. The thread will sleep in each 
> iteration for retry backoff ms.
>  # Async retries. In each polling, the retries do not meet the backoff will 
> be filtered. The data class often maintains a 1:1 mapping to a set of 
> requests which are logically associated. (i.e. a set contains only one 
> initial request and only its retries.)
> For type 1, we can utilize a local failure counter of a Java generic data 
> type.
> For case 2, I already wrapped the exponential backoff/timeout util class in 
> my KIP-601 
> [implementation|https://github.com/apache/kafka/pull/8683/files#diff-9ca2b1294653dfa914b9277de62b52e3R28]
>  which takes the number of attempts and returns the backoff/timeout value at 
> the corresponding level. Thus, we can add a new class property to those 
> classes containing retriable data in order to record the number of failed 
> attempts.
>  
> Changes:
> KafkaProducer:
>  # Produce request (ApiKeys.PRODUCE). Currently, the backoff applies to each 
> ProducerBatch in Accumulator, which already has an attribute attempts 
> recording the number of failed attempts. So we can let the Accumulator 
> calculate the new retry backoff for each bach when it enqueues them, to avoid 
> instantiate the util class multiple times.
>  # Transaction request (ApiKeys..*TXN). TxnRequestHandler will have a new 
> class property of type `Long` to record the number of attempts.
> KafkaConsumer:
>  # Some synchronous retry use cases. Record the failed attempts in the 
> blocking loop.
>  # Partition request (ApiKeys.OFFSET_FOR_LEADER_EPOCH, ApiKeys.LIST_OFFSETS). 
> Though the actual requests are packed for each node, the current 
> implementation is applying backoff to each topic partition, where the backoff 
> value is kept by TopicPartitionState. Thus, TopicPartitionState will have the 
> new property recording the number of attempts.
> Metadata:
>  #  Metadata lives as a singleton in many clients. Add a new property 
> recording the number of attempts
>  AdminClient:
>  # AdminClient has its own request abstraction Call. The failed attempts are 
> already kept by the abstraction. So probably clean the Call class logic a bit.
> Existing tests:
>  # If the tests are testing the retry backoff, add a delta to the assertion, 
> considering the existence of the jitter.
>  # If the tests are testing other functionality, we can specify the same 
> value for both `retry.backoff.ms` and `retry.backoff.max.ms` in order to make 
> the retry backoff static. We can use this trick to make the existing tests 
> compatible with the changes.
> There're other common usages look like client.poll(timeout), where the 
> timeout passed in is the retry backoff value. We won't change these usages 
> since its underlying logic is nioSelector.select(timeout) and 
> nioSelector.selectNow(), which means if no interested op exists, the client 
> will block retry backoff milliseconds. This is an optimization when there's 
> no request that needs to be sent but the client is waiting for responses. 
> Specifically, if the client fails the inflight requests before the retry 
> backoff milliseconds passed, it still needs to wait until that amount of time 
> passed, unless there's a new request need to be sent.
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2021-12-24 Thread Luke Chen (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17464944#comment-17464944
 ] 

Luke Chen commented on KAFKA-9800:
--

Creating 3 sub-tasks for this KIP, and co-authored with [~d8tltanc] . Thanks.

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Luke Chen
>Priority: Major
>  Labels: KIP-580
>
> Design:
> The main idea is to bookkeep the failed attempt. Currently, the retry backoff 
> has two main usage patterns:
>  # Synchronous retires and blocking loop. The thread will sleep in each 
> iteration for retry backoff ms.
>  # Async retries. In each polling, the retries do not meet the backoff will 
> be filtered. The data class often maintains a 1:1 mapping to a set of 
> requests which are logically associated. (i.e. a set contains only one 
> initial request and only its retries.)
> For type 1, we can utilize a local failure counter of a Java generic data 
> type.
> For case 2, I already wrapped the exponential backoff/timeout util class in 
> my KIP-601 
> [implementation|https://github.com/apache/kafka/pull/8683/files#diff-9ca2b1294653dfa914b9277de62b52e3R28]
>  which takes the number of attempts and returns the backoff/timeout value at 
> the corresponding level. Thus, we can add a new class property to those 
> classes containing retriable data in order to record the number of failed 
> attempts.
>  
> Changes:
> KafkaProducer:
>  # Produce request (ApiKeys.PRODUCE). Currently, the backoff applies to each 
> ProducerBatch in Accumulator, which already has an attribute attempts 
> recording the number of failed attempts. So we can let the Accumulator 
> calculate the new retry backoff for each bach when it enqueues them, to avoid 
> instantiate the util class multiple times.
>  # Transaction request (ApiKeys..*TXN). TxnRequestHandler will have a new 
> class property of type `Long` to record the number of attempts.
> KafkaConsumer:
>  # Some synchronous retry use cases. Record the failed attempts in the 
> blocking loop.
>  # Partition request (ApiKeys.OFFSET_FOR_LEADER_EPOCH, ApiKeys.LIST_OFFSETS). 
> Though the actual requests are packed for each node, the current 
> implementation is applying backoff to each topic partition, where the backoff 
> value is kept by TopicPartitionState. Thus, TopicPartitionState will have the 
> new property recording the number of attempts.
> Metadata:
>  #  Metadata lives as a singleton in many clients. Add a new property 
> recording the number of attempts
>  AdminClient:
>  # AdminClient has its own request abstraction Call. The failed attempts are 
> already kept by the abstraction. So probably clean the Call class logic a bit.
> Existing tests:
>  # If the tests are testing the retry backoff, add a delta to the assertion, 
> considering the existence of the jitter.
>  # If the tests are testing other functionality, we can specify the same 
> value for both `retry.backoff.ms` and `retry.backoff.max.ms` in order to make 
> the retry backoff static. We can use this trick to make the existing tests 
> compatible with the changes.
> There're other common usages look like client.poll(timeout), where the 
> timeout passed in is the retry backoff value. We won't change these usages 
> since its underlying logic is nioSelector.select(timeout) and 
> nioSelector.selectNow(), which means if no interested op exists, the client 
> will block retry backoff milliseconds. This is an optimization when there's 
> no request that needs to be sent but the client is waiting for responses. 
> Specifically, if the client fails the inflight requests before the retry 
> backoff milliseconds passed, it still needs to wait until that amount of time 
> passed, unless there's a new request need to be sent.
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2020-06-02 Thread Cheng Tan (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17124602#comment-17124602
 ] 

Cheng Tan commented on KAFKA-9800:
--

[~skaundinya] [~ijuma]

Thanks for the reply. Let's limit the patch scope to the original proposal. 

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Cheng Tan
>Priority: Major
>  Labels: KIP-580
>
> In {{KafkaAdminClient}}, we will have to modify the way the retry backoff is 
> calculated for the calls that have failed and need to be retried. >From the 
> current static retry backoff, we have to introduce a mechanism for all calls 
> that upon failure, the next retry time is dynamically calculated.



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


[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2020-06-02 Thread Ismael Juma (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17124354#comment-17124354
 ] 

Ismael Juma commented on KAFKA-9800:


For a bit more context, the plan was to add per request timeout support to 
NetworkClient and migrate AdminClient to use it. I had the start of a PR 
(([https://github.com/apache/kafka/pull/3503/|https://github.com/apache/kafka/pull/3503/files]),
 but we then decided to go with a simpler approach that was good enough for 
what the Consumer needed. Tackling that may increase the scope of this work by 
quite a bit though. It may be worth checking if we can encapsulate the logic 
for exponential backoff in separate classes and then use it from where retries 
are handled today for the relevant clients.

And tackle the consolidation of per request timeouts in a separate JIRA.

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Cheng Tan
>Priority: Major
>  Labels: KIP-580
>
> In {{KafkaAdminClient}}, we will have to modify the way the retry backoff is 
> calculated for the calls that have failed and need to be retried. >From the 
> current static retry backoff, we have to introduce a mechanism for all calls 
> that upon failure, the next retry time is dynamically calculated.



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


[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2020-06-02 Thread Sanjana Kaundinya (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17124340#comment-17124340
 ] 

Sanjana Kaundinya commented on KAFKA-9800:
--

Here are my thoughts:
1) There should be no changes to the underlying NetworkClient interface. More 
details on what interfaces should change are outlined in the KIP 
[here]([https://cwiki.apache.org/confluence/display/KAFKA/KIP-580%3A+Exponential+Backoff+for+Kafka+Clients)|https://cwiki.apache.org/confluence/display/KAFKA/KIP-580%3A+Exponential+Backoff+for+Kafka+Clients).].
 As [~d8tltanc] mentioned, the key difference between the AdminClient and the 
other clients is that the AdminClient supports a per request timeout whereas 
the Producer and Consumer are using the NetworkClient to handle the timeouts. 
In our implementation we should add a per-request support for all clients and 
reuse the code existing in CallRetryContext to apply to all clients to do this.

2) I agree with [~ijuma], I don't see really much of a benefit for doing a 
different backoff strategy per client, plus that kind of improvement goes 
beyond the scope of this KIP.

 

[~d8tltanc] could you shed more light on how to refactor the AdminClient to 
take out the redundant logic? I want to understand how much this would change 
and if all this change would still be in the scope of the KIP.

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Cheng Tan
>Priority: Major
>  Labels: KIP-580
>
> In {{KafkaAdminClient}}, we will have to modify the way the retry backoff is 
> calculated for the calls that have failed and need to be retried. >From the 
> current static retry backoff, we have to introduce a mechanism for all calls 
> that upon failure, the next retry time is dynamically calculated.



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


[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2020-06-02 Thread Ismael Juma (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17123935#comment-17123935
 ] 

Ismael Juma commented on KAFKA-9800:


I think you should apply the same backoff strategy for all request types. I 
don't see much benefit in the more complex approach.

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Cheng Tan
>Priority: Major
>  Labels: KIP-580
>
> In {{KafkaAdminClient}}, we will have to modify the way the retry backoff is 
> calculated for the calls that have failed and need to be retried. >From the 
> current static retry backoff, we have to introduce a mechanism for all calls 
> that upon failure, the next retry time is dynamically calculated.



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


[jira] [Commented] (KAFKA-9800) [KIP-580] Client Exponential Backoff Implementation

2020-06-02 Thread Cheng Tan (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17123377#comment-17123377
 ] 

Cheng Tan commented on KAFKA-9800:
--

Recap the discussion in Github. We want to implement a per-request backoff for 
all types of clients.

 

Let me talk about two of my major concerns and thoughts about implementing the 
universal client exponential backoff.

*AdminClient logic redundant*

The main request flow difference btw AdminClient and normal clients (e.g. 
Producer and Consumer) would be that AdminClient wants to have a per request 
timeout while normal clients is okay with a static default timeout. Thus, 
AdminClient rewrote a quite amount of NetworkClient's functionality.

For example,
 # Wrapping the request builder into a new class {{Call}}, (the construction 
lambda adds tons of lines into the AdminClient.java, which should probably have 
been living in each AbstractRequest implementation classes files)
 # Re-writing the request queues for different request status, while normal 
clients are fully using the NetworkClient.

These logics will become redundant after we support exponential backoff in 
NetworkClient for all types of clients. Are we considering refactoring the 
AdminClient further and remove all the redundant logic which should have 
belonged to the networking layer and the AbstractRequest implementation classes?

*Flexible backoff modes*

Let's analyze the request backoff demands of all the types of clients at this 
point. In my opinion, there are simply two:
 # Requests do not need exponential backoff. These requests need to be sent 
ASAP to avoid dataflow performance degradation, such as the {{ProduceRequest}} 
and its related/preceding metadata requests.

 # Request do need exponential backoff. These requests are “second-class 
citizens” and can be throttled to avoid request storms on the broker side. Such 
as metadata related requests in AdminClient.

Now the question comes. Even when two requests are of the same request type, 
one may have to get sent ASAP while the other one may wait, depending on the 
use case. We need to think deeper about how to make a classification.

But the implementation would be simple. We can utilize the existing builder 
pattern AbstractRequest and build the request flexibly upon a given 
retry_backoff mode. For example,
 # AbstractRequest.Builder will interact with a new abstract class specifying 
the retry_backoff option, static or exponential.
 # AbstractRequest will have some new interfaces controlling the backoff.

Then, we can control if the request should have a static backoff or an 
exponential backoff when we construct each implementation instance of 
AbstractRequest.Builder.

I'll include more details in the Jira ticket and rewrite this PR. Before we 
talk more about the code details and start the new implementation, please let 
me know what you think about the AdminClient refactor and static/exponential 
retry_backoff classification rule.

 

> [KIP-580] Client Exponential Backoff Implementation
> ---
>
> Key: KAFKA-9800
> URL: https://issues.apache.org/jira/browse/KAFKA-9800
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Cheng Tan
>Assignee: Cheng Tan
>Priority: Major
>  Labels: KIP-580
>
> In {{KafkaAdminClient}}, we will have to modify the way the retry backoff is 
> calculated for the calls that have failed and need to be retried. >From the 
> current static retry backoff, we have to introduce a mechanism for all calls 
> that upon failure, the next retry time is dynamically calculated.



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