One more change: as parameter is called "retries", default value should
be zero (instead of one).

-Matthias

On 11/10/17 1:14 PM, Matthias J. Sax wrote:
> Thanks for the feedback. Typos fixed.
> 
> Damian explained already why we need the new strategy.
> 
> @Kamal: many users don't want to retry but want to fail the Kafka Stream
> instance in case of an error. All default parameters are chosen to
> follow this pattern (similar to consumer/producer/broker defaults). The
> KIP aims to allow users to reconfigure Kafka Streams to be resilient
> against errors. It's a users choice to change configs to get better
> resilience.
> 
> 
> Update:
> 
> While I was working on the PR, I realized that parameter
> "retry.backoff.ms" is already available in StreamsConfig. I updated the
> KIP accordingly.
> 
> I also discovered, that we have a hard coded number of retries for state
> locks -- I think, it would be worth to reuse both parameters for those,
> too. WDYT?
> 
> Here is the current PR: https://github.com/apache/kafka/pull/4206
> 
> 
> -Matthias
> 
> 
> 
> On 11/9/17 2:29 PM, Guozhang Wang wrote:
>> Damian,
>>
>> You are right! I was dreaming at the wrong class :)
>>
>> Guozhang
>>
>>
>> On Thu, Nov 9, 2017 at 11:27 AM, Damian Guy <damian....@gmail.com> wrote:
>>
>>> Guozhang, i'm not sure i follow... Global stores aren't per task, they are
>>> per application instance and should be fully restored before the stream
>>> threads start processing. They don't go through a rebalance as it is manual
>>> assignment of all partitions in the topic.
>>>
>>> On Thu, 9 Nov 2017 at 17:43 Guozhang Wang <wangg...@gmail.com> wrote:
>>>
>>>> Instead of restoring the global store during registration, could we also
>>> do
>>>> this after the rebalance callback as in the main loop? By doing this we
>>> can
>>>> effectively swallow-and-retry-in-next-loop as we did for non-global
>>> stores.
>>>> Since global stores are per task not per thread, we would not process the
>>>> task after the global store is bootstrapped fully.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>> On Thu, Nov 9, 2017 at 7:08 AM, Bill Bejeck <bbej...@gmail.com> wrote:
>>>>
>>>>> Thanks for the KIP Matthias, +1 from me.
>>>>>
>>>>>
>>>>> -Bill
>>>>>
>>>>> On Thu, Nov 9, 2017 at 8:40 AM, Ted Yu <yuzhih...@gmail.com> wrote:
>>>>>
>>>>>> lgtm
>>>>>>
>>>>>> bq. pass both parameter
>>>>>>
>>>>>> parameter should be in plural.
>>>>>> Same with 'two new configuration parameter'
>>>>>>
>>>>>> Cheers
>>>>>>
>>>>>> On Thu, Nov 9, 2017 at 4:20 AM, Damian Guy <damian....@gmail.com>
>>>> wrote:
>>>>>>
>>>>>>> Thanks Matthias, LGTM
>>>>>>>
>>>>>>> On Thu, 9 Nov 2017 at 11:13 Matthias J. Sax <matth...@confluent.io
>>>>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> I want to propose a new KIP to make Streams API more resilient to
>>>>>> broker
>>>>>>>> disconnections.
>>>>>>>>
>>>>>>>>
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>> 224%3A+Add+configuration+parameters+%60retries%60+and+%
>>>>>>> 60retry.backoff.ms%60+to+Streams+API
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> -- Guozhang
>>>>
>>>
>>
>>
>>
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to