Matthias,

I try to clarify some concept.

When streams state is REBALANCING, it means the user can just plain retry.

When streams state is ERROR or PENDING_SHUTDOWN or NOT_RUNNING, it means
state store migrated to another instance, the user needs to rediscover the
store.

Is my understanding correct?


---
Vito

On Sun, Nov 5, 2017 at 12:30 AM, Matthias J. Sax <matth...@confluent.io>
wrote:

> Thanks for the KIP Vito!
>
> I agree with what Guozhang said. The original idea of the Jira was, to
> give different exceptions for different "recovery" strategies to the user.
>
> For example, if a store is currently recreated, a user just need to wait
> and can query the store later. On the other hand, if a store go migrated
> to another instance, a user needs to rediscover the store instead of a
> "plain retry".
>
> Fatal errors might be a third category.
>
> Not sure if there is something else?
>
> Anyway, the KIP should contain a section that talks about this ideas and
> reasoning.
>
>
> -Matthias
>
>
> On 11/3/17 11:26 PM, Guozhang Wang wrote:
> > Thanks for writing up the KIP.
> >
> > Vito, Matthias: one thing that I wanted to figure out first is what
> > categories of errors we want to notify the users, if we only wants to
> > distinguish fatal v.s. retriable then probably we should rename the
> > proposed StateStoreMigratedException / StateStoreClosedException classes.
> > And then from there we should list what are the possible internal
> > exceptions ever thrown in those APIs in the call trace, and which
> > exceptions should be wrapped to what others, and which ones should be
> > handled without re-throwing, and which ones should not be wrapped at all
> > but directly thrown to user's face.
> >
> > Guozhang
> >
> >
> > On Wed, Nov 1, 2017 at 11:09 PM, vito jeng <v...@is-land.com.tw> wrote:
> >
> >> Hi,
> >>
> >> I'd like to start discuss KIP-216:
> >>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> 216%3A+IQ+should+throw+different+exceptions+for+different+errors
> >>
> >> Please have a look.
> >> Thanks!
> >>
> >> ---
> >> Vito
> >>
> >
> >
> >
>
>

Reply via email to