You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Vito Jeng <vi...@is-land.com.tw> on 2019/08/09 02:53:06 UTC

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Thanks, Matthias!

> About `StreamThreadNotStartedException`:

Thank you for explanation. I agree with your opinion.
`CompositeReadOnlyXxxStore#get()` would never throw
`StreamThreadNotStartedException`.

For the case that corresponding thread crashes after we handed out the
store handle. We may throw `KafkaStreamsNotRunningException` or
`StateStoreMigratedException`.
In `StreamThreadStateStoreProvider`, we would throw
`KafkaStreamsNotRunningException` when stream thread is not running(
https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
store is closed(https://shorturl.at/hrvAN). So I think we do not need to
add a new type for this case. Does that make sense?


> About `KafkaStreamsNotRunningException` vs
`StreamThreadNotRunningException`:

I understand your point. I rename `StreamThreadNotRunningException` to
`KafkaStreamsNotRunningException`.


About check unknown state store names:
Thank you for the hint. I add a new type `UnknownStateStoreException` for
this case.


> Also, we should still have fatal exception
`StateStoreNotAvailableException`? Not sure why you remove it?

Thank you point this, already add it again.

The KIP already updated, please take a look.

---
Vito


On Tue, Jul 30, 2019 at 6:04 AM Matthias J. Sax <ma...@confluent.io>
wrote:

> Any update on this KIP Vito?
>
>
> On 7/11/19 4:26 PM, Matthias J. Sax wrote:
> > Thanks Vito! I think the KIP shapes out nicely!
> >
> >
> > To answer the open question you raised (I also adjust my answers based
> > on the latest KIP update)
> >
> >
> >
> > About `StreamThreadNotStartedException`: I understand what you pointed
> > out. However, I think we can consider the following: If a thread is not
> > started yet, and `KafkaStreams#store()` throw this exception, we would
> > not return a `CompositeReadOnlyXxxStore` to the user. Hence, `get()`
> > cannot be called. And if we return `CompositeReadOnlyXxxStore` the
> > thread was started and `get()` would never hit the condition to throw
> > the exception? Or do I miss something (this part of the logic is a
> > little tricky...)
> >
> > However, thinking about it, what could happen IMHO is, that the
> > corresponding thread crashes after we handed out the store handle. For
> > this case, it would make sense to throw an exception from `get()` but it
> > would be a different one IMHO. Maybe we need a new type
> > (`StreamThreadDeadException` or similar?) or we should reuse
> > `StoreMigratedException` because if a thread dies we would migrate the
> > store to another thread. (The tricky part might be, to detect this
> > condition correctly -- not 100% sure atm how we could do this.)
> >
> > What do you think about this?
> >
> >
> >
> > About `KafkaStreamsNotRunningException` vs
> > `StreamThreadNotRunningException` -- I see your point. Atm, I think we
> > don't allow querying at all if KafkaStreams is not in state RUNNING
> > (correct me if I am wrong). Hence, if there is an instance with 2
> > thread, and 1 thread is actually up and ready, but the other thread is
> > not, you cannot query anything. Only if both threads are in state
> > RUNNING we allow to query. It might be possible to change the code to
> > allow querying if a thread is ready independent from the other threads.
> > For this case, the name you suggest would make more sense. But I
> > _think_, that the current behavior is different and thus,
> > `KafkaStreamsNotRunningException` seems to reflect the current behavior
> > better? -- I also want to add that we are talking about a fatal
> > exception -- if a thread crashes, we would migrate the store to another
> > thread and it would not be fatal, but the store can be re-discovered.
> > Only if all thread would die, if would be fatal -- however, for this
> > case KafakStreams would transit to DEAD anyway.
> >
> >
> >
> >> When the user passes a store name to `KafkaStreams#store()`, does there
> >> have a way that distinguish the store name is "a wrong name" or
> "migrated"
> >> during `QueryableStoreProvider#getStore()`?
> >> From my current understanding, I cannot distinguish these two.
> >
> > This should be possible. In the private KafkaStreams constructor, we
> > have access to `InternalTopologyBuilder` that can give us all known
> > store names. Hence, we can get a set of all known store names, keep them
> > as a member variable and use in `KafkaStreams#store()` in an initial
> > check if the store name is valid or not.
> >
> >
> >
> >> Should we remove `StreamThreadNotRunningException` and throw
> >> `FatalStateStoreException` directly ?
> >
> > I would keep both, because `FatalStateStoreException` is not very
> > descriptive. Also, we should still have fatal exception
> > `StateStoreNotAvailableException`? Not sure why you remove it?
> >
> >
> >
> > Glad you found a way to avoid
> > `QueryableStoreType#setStreams(KafkaStreams streams)`.
> >
> >
> >
> > -Matthias
> >
> >
> > On 7/5/19 8:03 AM, Vito Jeng wrote:
> >> Hi, Mattias,
> >>
> >> Just completed the modification of KIP, please take a look when you are
> >> available.
> >>
> >> ---
> >> Vito
> >>
> >>
> >> On Wed, Jul 3, 2019 at 9:07 PM Vito Jeng <vi...@is-land.com.tw> wrote:
> >>
> >>> Hi, Matthias,
> >>>
> >>> This is second part.
> >>>
> >>>> For the internal exceptions:
> >>>>
> >>>> `StateStoreClosedException` -- why can it be wrapped as
> >>>> `StreamThreadNotStartedException` ? It seems that the later would only
> >>>> be thrown by `KafkaStreams#store()` and thus would be throw directly.
> >>>
> >>> Both `StateStoreClosedException` and `EmptyStateStoreException` not
> can be
> >>> wrapped as `StreamThreadNotStartedException`.
> >>> This is a mistaken written in the previous KIP. Thank you point this.
> >>>
> >>>> A closed-exception should only happen after a store was successfully
> >>>> retrieved but cannot be queried any longer? Hence, converting/wrapping
> >>>> it into a `StateStoreMigratedException` make sense. I am also not
> sure,
> >>>> when a closed-exception would be wrapped by a
> >>>> `StateStoreNotAvailableException` (implying my understanding as
> describe
> >>>> above)?
> >>>>
> >>>> Same questions about `EmptyStateStoreException`.
> >>>>
> >>>> Thinking about both internal exceptions twice, I am wondering if it
> >>>> makes sense to have both internal exceptions at all? I have the
> >>>> impression that it make only sense to wrap them with a
> >>>> `StateStoreMigragedException`, but if they are wrapped into the same
> >>>> exception all the time, we can just remove both and throw
> >>>> `StateStoreMigratedException` directly?
> >>>
> >>> After deeper thinking, I think you are right. It seems we can throw
> >>> `StateStoreMigratedException` directly.
> >>> So that we can remove `StateStoreClosedException`,
> >>> `EmptyStateStoreException` and `StateStoreNotAvailableException`.
> >>> Will update the KIP.
> >>>
> >>> BTW, if we remove above three exceptions, the
> >>> `StreamThreadNotRunningException` will be the only one sub class
> extends
> >>> from FatalStateStoreException.
> >>> Should we remove `StreamThreadNotRunningException` and throw
> >>> `FatalStateStoreException` directly ?
> >>>
> >>>> Last point: Why do we need to add?
> >>>> QueryableStoreType#setStreams(KafkaStreams streams);
> >>>> John asked this question already and you replied to it. But I am not
> >>>> sure what your answer means. Can you explain it in more detail?
> >>>
> >>> The main purpose is to pass the KafkaStreams reference into
> >>> CompositeReadOnlyKeyValueStore / CompositeReadOnlySessionStore/
> >>> CompositeReadOnlyWindowStore instance.
> >>> We need check KafkaStreams state to warp InvalidStateStoreException in
> to
> >>> other exception(e.g., StateStoreMigratedException) when the user
> accesses
> >>> these read-only stores.
> >>>
> >>> The original thought is to add `setStreams` method in to
> >>> QueryableStoreType. But now I think I find a better way during recent
> days.
> >>> This way does not need to change any public interface. So we can skip
> this
> >>> question. :)
> >>>
> >>>
> >>> I will update the KIP based on our discussion.
> >>> Thank you for help to finish the KIP!
> >>>
> >>> ---
> >>> Vito
> >>>
> >>>
> >>> On Thu, Jun 6, 2019 at 8:23 AM Matthias J. Sax <ma...@confluent.io>
> >>> wrote:
> >>>
> >>>> Hi Vito,
> >>>>
> >>>> sorry for dropping this discussion on the floor a while back. I was
> just
> >>>> re-reading the KIP and discussion thread, and I think it is shaping
> out
> >>>> nicely!
> >>>>
> >>>> I like the overall hierarchy of the exception classes.
> >>>>
> >>>> Some things are still not 100% clear:
> >>>>
> >>>>
> >>>> You listed all methods that may throw an `InvalidStateStoreException`
> >>>> atm. For the new exceptions, can any exception be thrown by any
> method?
> >>>> It might help to understand this relationship better.
> >>>>
> >>>> For example, StreamThreadNotStartedException, seems to only make sense
> >>>> for `KafkaStreams#store()`?
> >>>>
> >>>>
> >>>> For `StreamThreadNotRunningException` should we rename it to
> >>>> `KafkaStreamsNotRunningException` ?
> >>>>
> >>>>
> >>>> The description of `StreamThreadNotRunningException` and
> >>>> `StateStoreNotAvailableException` seems to be the same? From my
> >>>> understandng, the description makes sense for
> >>>> `StreamThreadNotRunningException` -- for
> >>>> `StateStoreNotAvailableException` I was expecting/inferring from the
> >>>> name, that it would be thrown if no such store exists in the topology
> at
> >>>> all (ie, user passed in a invalid/wrong store name). For this case,
> this
> >>>> exception should be thrown only from `KafkaStreams#store()` ?
> >>>>
> >>>>
> >>>> For the internal exceptions:
> >>>>
> >>>> `StateStoreClosedException` -- why can it be wrapped as
> >>>> `StreamThreadNotStartedException` ? It seems that the later would only
> >>>> be thrown by `KafkaStreams#store()` and thus would be throw directly.
> A
> >>>> closed-exception should only happen after a store was successfully
> >>>> retrieved but cannot be queried any longer? Hence, converting/wrapping
> >>>> it into a `StateStoreMigratedException` make sense. I am also not
> sure,
> >>>> when a closed-exception would be wrapped by a
> >>>> `StateStoreNotAvailableException` (implying my understanding as
> describe
> >>>> above)?
> >>>>
> >>>> Same questions about `EmptyStateStoreException`.
> >>>>
> >>>> Thinking about both internal exceptions twice, I am wondering if it
> >>>> makes sense to have both internal exceptions at all? I have the
> >>>> impression that it make only sense to wrap them with a
> >>>> `StateStoreMigragedException`, but if they are wrapped into the same
> >>>> exception all the time, we can just remove both and throw
> >>>> `StateStoreMigratedException` directly?
> >>>>
> >>>>
> >>>> Last point: Why do we need to add?
> >>>>
> >>>>> QueryableStoreType#setStreams(KafkaStreams streams);
> >>>>
> >>>> John asked this question already and you replied to it. But I am not
> >>>> sure what your answer means. Can you explain it in more detail?
> >>>>
> >>>>
> >>>>
> >>>> Thanks for your patience on this KIP!
> >>>>
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On 11/11/18 4:55 AM, Vito Jeng wrote:
> >>>>> Hi, Matthias,
> >>>>>
> >>>>> KIP already updated.
> >>>>>
> >>>>>> - StateStoreClosedException:
> >>>>>>   will be wrapped to StateStoreMigratedException or
> >>>>> StateStoreNotAvailableException later.
> >>>>>> Can you clarify the cases (ie, when will it be wrapped with the one
> or
> >>>>> the other)?
> >>>>>
> >>>>> For example, in the
> implementation(CompositeReadOnlyKeyValueStore#get),
> >>>> we
> >>>>> get all stores first, and then call ReadOnlyKeyValueStore#get to get
> >>>> value
> >>>>> in every store iteration.
> >>>>>
> >>>>> When calling ReadOnlyKeyValueStore#get, the StateStoreClosedException
> >>>> will
> >>>>> be thrown if the state store is not open.
> >>>>> We need catch StateStoreClosedException and wrap it in different
> >>>> exception
> >>>>> type:
> >>>>>   * If the stream's state is CREATED, we wrap
> StateStoreClosedException
> >>>>> with StreamThreadNotStartedException. User can retry until to
> RUNNING.
> >>>>>   * If the stream's state is RUNNING / REBALANCING, the state store
> >>>> should
> >>>>> be migrated, we wrap StateStoreClosedException with
> >>>>> StateStoreMigratedException. User can rediscover the state store.
> >>>>>   * If the stream's state is PENDING_SHUTDOWN / NOT_RUNNING / ERROR,
> the
> >>>>> stream thread is not available, we wrap StateStoreClosedException
> with
> >>>>> StateStoreNotAvailableException. User cannot retry when this
> exception
> >>>> is
> >>>>> thrown.
> >>>>>
> >>>>>
> >>>>>> - StateStoreIsEmptyException:
> >>>>>>  I don't understand the semantic of this exception. Maybe it's a
> naming
> >>>>> issue?
> >>>>>
> >>>>> I think yes. :)
> >>>>> Does `EmptyStateStoreException` is better ? (already updated in the
> KIP)
> >>>>>
> >>>>>
> >>>>>> - StateStoreIsEmptyException:
> >>>>>> will be wrapped to StateStoreMigratedException or
> >>>>> StateStoreNotAvailableException later.
> >>>>>> Also, can you clarify the cases (ie, when will it be wrapped with
> the
> >>>> one
> >>>>> or the other)?
> >>>>>
> >>>>> For example, in the implementation
> >>>> (CompositeReadOnlyKeyValueStore#get), we
> >>>>> call StateStoreProvider#stores (WrappingStoreProvider#stores) to get
> all
> >>>>> stores. EmptyStateStoreException will be thrown when cannot find any
> >>>> store
> >>>>> and then we need catch it and wrap it in different exception type:
> >>>>>   * If the stream's state is CREATED, we wrap
> EmptyStateStoreException
> >>>> with
> >>>>> StreamThreadNotStartedException. User can retry until to RUNNING.
> >>>>>   * If the stream's state is RUNNING / REBALANCING, the state store
> >>>> should
> >>>>> be migrated, we wrap EmptyStateStoreException with
> >>>>> StateStoreMigratedException. User can rediscover the state store.
> >>>>>   * If the stream's state is PENDING_SHUTDOWN / NOT_RUNNING / ERROR,
> the
> >>>>> stream thread is not available, we wrap EmptyStateStoreException with
> >>>>> StateStoreNotAvailableException. User cannot retry when this
> exception
> >>>> is
> >>>>> thrown.
> >>>>>
> >>>>> I hope the above reply can clarify.
> >>>>>
> >>>>> The last one that was not replied was:
> >>>>>
> >>>>>> I am also wondering, if we should introduce a fatal exception
> >>>>>> `UnkownStateStoreException` to tell users that they passed in an
> >>>> unknown
> >>>>>> store name?
> >>>>>
> >>>>> Until now, unknown state store is not thinking about in the KIP.
> >>>>> I believe it would be very useful for users.
> >>>>>
> >>>>> Looking at the related code(WrappingStoreProvider#stores),
> >>>>> I found that I can't distinguish between the state store was migrated
> >>>> or an
> >>>>> unknown state store.
> >>>>>
> >>>>> Any thoughts?
> >>>>>
> >>>>> ---
> >>>>> Vito
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Sun, Nov 11, 2018 at 5:31 PM Vito Jeng <vi...@is-land.com.tw>
> wrote:
> >>>>>
> >>>>>> Hi, Matthias,
> >>>>>>
> >>>>>> Sorry for the late reply.
> >>>>>>
> >>>>>>> I am wondering what the semantic impact/change is, if we introduce
> >>>>>>> `RetryableStateStoreException` and `FatalStateStoreException` that
> >>>> both
> >>>>>>> inherit from it. While I like the introduction of both from a high
> >>>> level
> >>>>>>> point of view, I just want to make sure it's semantically sound and
> >>>>>>> backward compatible. Atm, I think it's fine, but I want to point it
> >>>> out
> >>>>>>> such that everybody can think about this, too, so we can verify
> that
> >>>>>>> it's a natural evolving API change.
> >>>>>>
> >>>>>> Thank you for pointing this out. This's really important for public
> >>>> API.
> >>>>>>
> >>>>>> Just when I was replying to you, I found that KIP needs some modify.
> >>>>>> I will fix it ASAP, and then let's continue the discussion.
> >>>>>>
> >>>>>> ---
> >>>>>> Vito
> >>>>>>
> >>>>>>
> >>>>>> On Wed, Nov 7, 2018 at 7:06 AM Matthias J. Sax <
> matthias@confluent.io>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hey Vito,
> >>>>>>>
> >>>>>>> I saw that you updated your PR, but did not reply to my last
> comments.
> >>>>>>> Any thoughts?
> >>>>>>>
> >>>>>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>> On 10/19/18 10:34 AM, Matthias J. Sax wrote:
> >>>>>>>> Glad to have you back Vito :)
> >>>>>>>>
> >>>>>>>> Some follow up thoughts:
> >>>>>>>>
> >>>>>>>>  - the current `InvalidStateStoreException` is documents as being
> >>>>>>>> sometimes retry-able. From the JavaDocs:
> >>>>>>>>
> >>>>>>>>> These exceptions may be transient [...] Hence, it is valid to
> >>>> backoff
> >>>>>>> and retry when handling this exception.
> >>>>>>>>
> >>>>>>>> I am wondering what the semantic impact/change is, if we introduce
> >>>>>>>> `RetryableStateStoreException` and `FatalStateStoreException` that
> >>>> both
> >>>>>>>> inherit from it. While I like the introduction of both from a high
> >>>> level
> >>>>>>>> point of view, I just want to make sure it's semantically sound
> and
> >>>>>>>> backward compatible. Atm, I think it's fine, but I want to point
> it
> >>>> out
> >>>>>>>> such that everybody can think about this, too, so we can verify
> that
> >>>>>>>> it's a natural evolving API change.
> >>>>>>>>
> >>>>>>>>  - StateStoreClosedException:
> >>>>>>>>
> >>>>>>>>> will be wrapped to StateStoreMigratedException or
> >>>>>>> StateStoreNotAvailableException later.
> >>>>>>>>
> >>>>>>>> Can you clarify the cases (ie, when will it be wrapped with the
> one
> >>>> or
> >>>>>>>> the other)?
> >>>>>>>>
> >>>>>>>>  - StateStoreIsEmptyException:
> >>>>>>>>
> >>>>>>>> I don't understand the semantic of this exception. Maybe it's a
> >>>> naming
> >>>>>>>> issue?
> >>>>>>>>
> >>>>>>>>> will be wrapped to StateStoreMigratedException or
> >>>>>>> StateStoreNotAvailableException later.
> >>>>>>>>
> >>>>>>>> Also, can you clarify the cases (ie, when will it be wrapped with
> the
> >>>>>>>> one or the other)?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> I am also wondering, if we should introduce a fatal exception
> >>>>>>>> `UnkownStateStoreException` to tell users that they passed in an
> >>>> unknown
> >>>>>>>> store name?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 10/17/18 8:14 PM, vito jeng wrote:
> >>>>>>>>> Just open a PR for further discussion:
> >>>>>>>>> https://github.com/apache/kafka/pull/5814
> >>>>>>>>>
> >>>>>>>>> Any suggestion is welcome.
> >>>>>>>>> Thanks!
> >>>>>>>>>
> >>>>>>>>> ---
> >>>>>>>>> Vito
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, Oct 11, 2018 at 12:14 AM vito jeng <vi...@is-land.com.tw>
> >>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi John,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for reviewing the KIP.
> >>>>>>>>>>
> >>>>>>>>>>> I didn't follow the addition of a new method to the
> >>>>>>> QueryableStoreType
> >>>>>>>>>>> interface. Can you elaborate why this is necessary to support
> the
> >>>> new
> >>>>>>>>>>> exception types?
> >>>>>>>>>>
> >>>>>>>>>> To support the new exception types, I would check stream state
> in
> >>>> the
> >>>>>>>>>> following classes:
> >>>>>>>>>>   - CompositeReadOnlyKeyValueStore class
> >>>>>>>>>>   - CompositeReadOnlySessionStore class
> >>>>>>>>>>   - CompositeReadOnlyWindowStore class
> >>>>>>>>>>   - DelegatingPeekingKeyValueIterator class
> >>>>>>>>>>
> >>>>>>>>>> It is also necessary to keep backward compatibility. So I plan
> >>>> passing
> >>>>>>>>>> stream
> >>>>>>>>>> instance to QueryableStoreType instance during
> KafkaStreams#store()
> >>>>>>>>>> invoked.
> >>>>>>>>>> It looks a most simple way, I think.
> >>>>>>>>>>
> >>>>>>>>>> It is why I add a new method to the QueryableStoreType
> interface. I
> >>>>>>> can
> >>>>>>>>>> understand
> >>>>>>>>>> that we should try to avoid adding the public api method.
> However,
> >>>> at
> >>>>>>> the
> >>>>>>>>>> moment
> >>>>>>>>>> I have no better ideas.
> >>>>>>>>>>
> >>>>>>>>>> Any thoughts?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>> Also, looking over your KIP again, it seems valuable to
> introduce
> >>>>>>>>>>> "retriable store exception" and "fatal store exception" marker
> >>>>>>> interfaces
> >>>>>>>>>>> that the various exceptions can mix in. It would be nice from a
> >>>>>>> usability
> >>>>>>>>>>> perspective to be able to just log and retry on any "retriable"
> >>>>>>> exception
> >>>>>>>>>>> and log and shutdown on any fatal exception.
> >>>>>>>>>>
> >>>>>>>>>> I agree that this is valuable to the user.
> >>>>>>>>>> I'll update the KIP.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thanks
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> ---
> >>>>>>>>>> Vito
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Oct 9, 2018 at 2:30 AM John Roesler <jo...@confluent.io>
> >>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Vito,
> >>>>>>>>>>>
> >>>>>>>>>>> I'm glad to hear you're well again!
> >>>>>>>>>>>
> >>>>>>>>>>> I didn't follow the addition of a new method to the
> >>>>>>> QueryableStoreType
> >>>>>>>>>>> interface. Can you elaborate why this is necessary to support
> the
> >>>> new
> >>>>>>>>>>> exception types?
> >>>>>>>>>>>
> >>>>>>>>>>> Also, looking over your KIP again, it seems valuable to
> introduce
> >>>>>>>>>>> "retriable store exception" and "fatal store exception" marker
> >>>>>>> interfaces
> >>>>>>>>>>> that the various exceptions can mix in. It would be nice from a
> >>>>>>> usability
> >>>>>>>>>>> perspective to be able to just log and retry on any "retriable"
> >>>>>>> exception
> >>>>>>>>>>> and log and shutdown on any fatal exception.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> -John
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Oct 5, 2018 at 11:47 AM Guozhang Wang <
> wangguoz@gmail.com
> >>>>>
> >>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Thanks for the explanation, that makes sense.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Mon, Jun 25, 2018 at 2:28 PM, Matthias J. Sax <
> >>>>>>> matthias@confluent.io
> >>>>>>>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> The scenario I had I mind was, that KS is started in one
> thread
> >>>>>>> while
> >>>>>>>>>>> a
> >>>>>>>>>>>>> second thread has a reference to the object to issue queries.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> If a query is issue before the "main thread" started KS, and
> the
> >>>>>>>>>>> "query
> >>>>>>>>>>>>> thread" knows that it will eventually get started, it can
> >>>> retry. On
> >>>>>>>>>>> the
> >>>>>>>>>>>>> other hand, if KS is in state PENDING_SHUTDOWN or DEAD, it is
> >>>>>>>>>>> impossible
> >>>>>>>>>>>>> to issue any query against it now or in the future and thus
> the
> >>>>>>> error
> >>>>>>>>>>> is
> >>>>>>>>>>>>> not retryable.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 6/25/18 10:15 AM, Guozhang Wang wrote:
> >>>>>>>>>>>>>> I'm wondering if StreamThreadNotStarted could be merged into
> >>>>>>>>>>>>>> StreamThreadNotRunning, because I think users' handling
> logic
> >>>> for
> >>>>>>>>>>> the
> >>>>>>>>>>>>> third
> >>>>>>>>>>>>>> case would be likely the same as the second. Do you have
> some
> >>>>>>>>>>> scenarios
> >>>>>>>>>>>>>> where users may want to handle them differently?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Sun, Jun 24, 2018 at 5:25 PM, Matthias J. Sax <
> >>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Sorry to hear! Get well soon!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> It's not a big deal if the KIP stalls a little bit. Feel
> free
> >>>> to
> >>>>>>>>>>> pick
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>>> up again when you find time.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Is `StreamThreadNotRunningException` really an retryable
> >>>>>>> error?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> When KafkaStream state is REBALANCING, I think it is a
> >>>>>>> retryable
> >>>>>>>>>>>>> error.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> StreamThreadStateStoreProvider#stores() will throw
> >>>>>>>>>>>>>>>>> StreamThreadNotRunningException when StreamThread state
> is
> >>>> not
> >>>>>>>>>>>>>>> RUNNING. The
> >>>>>>>>>>>>>>>>> user can retry until KafkaStream state is RUNNING.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I see. If this is the intention, than I would suggest to
> have
> >>>> two
> >>>>>>>>>>> (or
> >>>>>>>>>>>>>>> maybe three) different exceptions:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>  - StreamThreadRebalancingException (retryable)
> >>>>>>>>>>>>>>>  - StreamThreadNotRunning (not retryable -- thrown if in
> state
> >>>>>>>>>>>>>>> PENDING_SHUTDOWN or DEAD
> >>>>>>>>>>>>>>>  - maybe StreamThreadNotStarted (for state CREATED)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The last one is tricky and could also be merged into one of
> >>>> the
> >>>>>>>>>>> first
> >>>>>>>>>>>>>>> two, depending if you want to argue that it's retryable or
> >>>> not.
> >>>>>>>>>>> (Just
> >>>>>>>>>>>>>>> food for though -- not sure what others think.)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 6/22/18 8:06 AM, vito jeng wrote:
> >>>>>>>>>>>>>>>> Matthias,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thank you for your assistance.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> what is the status of this KIP?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Unfortunately, there is no further progress.
> >>>>>>>>>>>>>>>> About seven weeks ago, I was injured in sports. I had a
> >>>> broken
> >>>>>>>>>>> wrist
> >>>>>>>>>>>> on
> >>>>>>>>>>>>>>>> my left wrist.
> >>>>>>>>>>>>>>>> Many jobs are affected, including this KIP and
> >>>> implementation.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I just re-read it, and have a couple of follow up
> comments.
> >>>> Why
> >>>>>>>>>>> do
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> discuss the internal exceptions you want to add? Also,
> do we
> >>>>>>>>>>> really
> >>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>> them? Can't we just throw the correct exception directly
> >>>>>>> instead
> >>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> wrapping it later?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I think you may be right. As I say in the previous:
> >>>>>>>>>>>>>>>> "The original idea is that we can distinguish different
> state
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>>> exception for different handling. But to be honest, I am
> not
> >>>>>>> quite
> >>>>>>>>>>>> sure
> >>>>>>>>>>>>>>>> this is necessary. Maybe have some change during
> >>>>>>> implementation."
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> During the implementation, I also feel we maybe not need
> >>>> wrapper
> >>>>>>>>>>> it.
> >>>>>>>>>>>>>>>> We can just throw the correctly directly.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Is `StreamThreadNotRunningException` really an retryable
> >>>> error?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> When KafkaStream state is REBALANCING, I think it is a
> >>>> retryable
> >>>>>>>>>>>> error.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> StreamThreadStateStoreProvider#stores() will throw
> >>>>>>>>>>>>>>>> StreamThreadNotRunningException when StreamThread state is
> >>>> not
> >>>>>>>>>>>>> RUNNING.
> >>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>> user can retry until KafkaStream state is RUNNING.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> When would we throw an `StateStoreEmptyException`? The
> >>>>>>> semantics
> >>>>>>>>>>> is
> >>>>>>>>>>>>>>>> unclear to me atm.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> When the state is RUNNING, is
> `StateStoreClosedException` a
> >>>>>>>>>>>> retryable
> >>>>>>>>>>>>>>>> error?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> These two comments will be answered in another mail.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Mon, Jun 11, 2018 at 8:12 AM, Matthias J. Sax <
> >>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Vito,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> what is the status of this KIP?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I just re-read it, and have a couple of follow up
> comments.
> >>>> Why
> >>>>>>>>>>> do
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> discuss the internal exceptions you want to add? Also,
> do we
> >>>>>>>>>>> really
> >>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>> them? Can't we just throw the correct exception directly
> >>>>>>> instead
> >>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> wrapping it later?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> When would we throw an `StateStoreEmptyException`? The
> >>>>>>> semantics
> >>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> unclear to me atm.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Is `StreamThreadNotRunningException` really an retryable
> >>>> error?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> When the state is RUNNING, is
> `StateStoreClosedException` a
> >>>>>>>>>>>> retryable
> >>>>>>>>>>>>>>>>> error?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> One more nits: ReadOnlyWindowStore got a new method
> #fetch(K
> >>>>>>> key,
> >>>>>>>>>>>> long
> >>>>>>>>>>>>>>>>> time); that should be added
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Overall I like the KIP but some details are still
> unclear.
> >>>>>>> Maybe
> >>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> might help if you open an PR in parallel?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 4/24/18 8:18 AM, vito jeng wrote:
> >>>>>>>>>>>>>>>>>> Hi, Guozhang,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks for the comment!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi, Bill,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I'll try to make some update to make the KIP better.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks for the comment!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Sat, Apr 21, 2018 at 5:40 AM, Bill Bejeck <
> >>>>>>> bbejeck@gmail.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi Vito,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for the KIP, overall it's a +1 from me.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> At this point, the only thing I would change is
> possibly
> >>>>>>>>>>> removing
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> listing of all methods called by the user and the
> listing
> >>>> of
> >>>>>>>>>>> all
> >>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>> types and focus on what states result in which
> exceptions
> >>>>>>>>>>> thrown
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> user.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Fri, Apr 20, 2018 at 2:10 PM, Guozhang Wang <
> >>>>>>>>>>>> wangguoz@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for the KIP Vito!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I made a pass over the wiki and it looks great to me.
> >>>> I'm +1
> >>>>>>>>>>> on
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> About the base class InvalidStateStoreException
> itself,
> >>>> I'd
> >>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>> suggest we do not deprecate it but still expose it as
> >>>> part
> >>>>>>> of
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> public
> >>>>>>>>>>>>>>>>>>>> API, for people who do not want to handle these cases
> >>>>>>>>>>> differently
> >>>>>>>>>>>>> (if
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> deprecate it then we are enforcing them to capture all
> >>>> three
> >>>>>>>>>>>>>>> exceptions
> >>>>>>>>>>>>>>>>>>>> one-by-one).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Fri, Apr 20, 2018 at 9:14 AM, John Roesler <
> >>>>>>>>>>> john@confluent.io
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Vito,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP!
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I think it's much nicer to give callers different
> >>>>>>> exceptions
> >>>>>>>>>>> to
> >>>>>>>>>>>>> tell
> >>>>>>>>>>>>>>>>>>> them
> >>>>>>>>>>>>>>>>>>>>> whether the state store got migrated, whether it's
> still
> >>>>>>>>>>>>>>> initializing,
> >>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>> whether there's some unrecoverable error.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> In the KIP, it's typically not necessary to discuss
> >>>>>>>>>>>>> non-user-facing
> >>>>>>>>>>>>>>>>>>>> details
> >>>>>>>>>>>>>>>>>>>>> such as what exceptions we will throw internally. The
> >>>> KIP
> >>>>>>> is
> >>>>>>>>>>>>>>> primarily
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> discuss public interface changes.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> You might consider simply removing all the internal
> >>>> details
> >>>>>>>>>>> from
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> KIP,
> >>>>>>>>>>>>>>>>>>>>> which will have the dual advantage that it makes the
> KIP
> >>>>>>>>>>> smaller
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> easier
> >>>>>>>>>>>>>>>>>>>>> to agree on, as well as giving you more freedom in
> the
> >>>>>>>>>>> internal
> >>>>>>>>>>>>>>>>> details
> >>>>>>>>>>>>>>>>>>>>> when it comes to implementation.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I like your decision to have your refined exceptions
> >>>> extend
> >>>>>>>>>>>>>>>>>>>>> InvalidStateStoreException to ensure backward
> >>>>>>> compatibility.
> >>>>>>>>>>>> Since
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>> to encourage callers to catch the more specific
> >>>> exceptions,
> >>>>>>>>>>> and
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>> expect to ever throw a raw InvalidStateStoreException
> >>>>>>>>>>> anymore,
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>>>> consider adding the @Deprecated annotation to
> >>>>>>>>>>>>>>>>>>> InvalidStateStoreException.
> >>>>>>>>>>>>>>>>>>>>> This will gently encourage callers to migrate to the
> new
> >>>>>>>>>>>> exception
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> open
> >>>>>>>>>>>>>>>>>>>>> the possibility of removing
> InvalidStateStoreException
> >>>>>>>>>>> entirely
> >>>>>>>>>>>>> in a
> >>>>>>>>>>>>>>>>>>>> future
> >>>>>>>>>>>>>>>>>>>>> release.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Thu, Apr 19, 2018 at 8:58 AM, Matthias J. Sax <
> >>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for clarification! That makes sense to me.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Can you update the KIP to make those suggestions
> >>>> explicit?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 4/18/18 2:19 PM, vito jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>> Matthias,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks for the feedback!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> It's up to you to keep the details part in the
> KIP or
> >>>>>>> not.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Got it!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> The (incomplete) question was, if we need
> >>>>>>>>>>>>>>>>>>> `StateStoreFailException`
> >>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>> if existing `InvalidStateStoreException` could be
> >>>> used?
> >>>>>>> Do
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>>>>>>> that `InvalidStateStoreException` is not thrown at
> >>>> all
> >>>>>>>>>>>> anymore,
> >>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>> the new sub-classes (just to get a better
> >>>>>>> understanding).
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Yes. I suggest that `InvalidStateStoreException` is
> >>>> not
> >>>>>>>>>>> thrown
> >>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>> anymore,
> >>>>>>>>>>>>>>>>>>>>>>> but only new sub-classes.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Not sure what this sentence means:
> >>>>>>>>>>>>>>>>>>>>>>>>> The internal exception will be wrapped as
> category
> >>>>>>>>>>> exception
> >>>>>>>>>>>>>>>>>>>> finally.
> >>>>>>>>>>>>>>>>>>>>>>>> Can you elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> For example,
> `StreamThreadStateStoreProvider#stores()`
> >>>>>>> will
> >>>>>>>>>>>>> throw
> >>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`(internal
> exception).
> >>>>>>>>>>>>>>>>>>>>>>> And then the internal exception will be wrapped as
> >>>>>>>>>>>>>>>>>>>>>>> `StateStoreRetryableException` or
> >>>>>>> `StateStoreFailException`
> >>>>>>>>>>>>> during
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> `KafkaStreams.store()` and throw to the user.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Can you explain the purpose of the "internal
> >>>>>>> exceptions".
> >>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>> unclear
> >>>>>>>>>>>>>>>>>>>>>>> to me atm why they are introduced.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hmmm...the purpose of the "internal exceptions" is
> to
> >>>>>>>>>>>>> distinguish
> >>>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>>>>>>> the different kinds of InvalidStateStoreException.
> >>>>>>>>>>>>>>>>>>>>>>> The original idea is that we can distinguish
> different
> >>>>>>>>>>> state
> >>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>> exception for
> >>>>>>>>>>>>>>>>>>>>>>> different handling.
> >>>>>>>>>>>>>>>>>>>>>>> But to be honest, I am not quite sure this is
> >>>> necessary.
> >>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>> some change during implementation.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Does it make sense?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Mon, Apr 16, 2018 at 5:59 PM, Matthias J. Sax <
> >>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the update Vito!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> It's up to you to keep the details part in the
> KIP or
> >>>>>>> not.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> The (incomplete) question was, if we need
> >>>>>>>>>>>>>>>>>>> `StateStoreFailException`
> >>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>> if existing `InvalidStateStoreException` could be
> >>>> used?
> >>>>>>> Do
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>>>>>>> that `InvalidStateStoreException` is not thrown at
> >>>> all
> >>>>>>>>>>>> anymore,
> >>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>> the new sub-classes (just to get a better
> >>>>>>> understanding).
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Not sure what this sentence means:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> The internal exception will be wrapped as
> category
> >>>>>>>>>>> exception
> >>>>>>>>>>>>>>>>>>>> finally.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Can you elaborate?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Can you explain the purpose of the "internal
> >>>>>>> exceptions".
> >>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>> unclear
> >>>>>>>>>>>>>>>>>>>>>>>> to me atm why they are introduced.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On 4/10/18 12:33 AM, vito jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>> Matthias,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the review.
> >>>>>>>>>>>>>>>>>>>>>>>>> I reply separately in the following sections.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Sun, Apr 8, 2018 at 1:30 PM, Matthias J. Sax <
> >>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP and sorry for the
> long
> >>>>>>>>>>> pause...
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Seems you did a very thorough investigation of
> the
> >>>>>>> code.
> >>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>> useful
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> understand what user facing interfaces are
> >>>> affected.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> (Some parts might be even too detailed for a
> KIP.)
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I also think too detailed. Especially the section
> >>>>>>>>>>> `Changes
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>> trace`.
> >>>>>>>>>>>>>>>>>>>>>>>>> Do you think it should be removed?
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> To summarize my current understanding of your
> KIP,
> >>>> the
> >>>>>>>>>>> main
> >>>>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> introduce new exceptions that extend
> >>>>>>>>>>>>>>>>>>> `InvalidStateStoreException`.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> yep. Keep compatibility in this KIP is important
> >>>>>>> things.
> >>>>>>>>>>>>>>>>>>>>>>>>> I think the best way is that all new exceptions
> >>>> extend
> >>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>> `InvalidStateStoreException`.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Some questions:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>  - Why do we need ```? Could
> >>>>>>>>>>> `InvalidStateStoreException`
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> used
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>> this purpose?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Does this question miss some word?
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>  - What the superclass of
> >>>>>>>>>>> `StateStoreStreamThreadNotRunni
> >>>>>>>>>>>>>>>>>>>>> ngException`
> >>>>>>>>>>>>>>>>>>>>>>>>>> is? Should it be `InvalidStateStoreException` or
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreFailException`
> >>>>>>>>>>>>>>>>>>>>>>>>>> ?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>  - Is `StateStoreClosed` a fatal or retryable
> >>>>>>> exception
> >>>>>>>>>>> ?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I apologize for not well written parts. I tried
> to
> >>>>>>> modify
> >>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> recent period and modify the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>> The modification is now complete. Please look
> again.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On 2/21/18 5:15 PM, vito jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Sorry for not response these days.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I just finished it. Please have a look. :)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Feb 14, 2018 at 5:45 AM, Matthias J.
> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is there any update on this KIP?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 1/3/18 12:59 AM, vito jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your response.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think you are right. We need to look at the
> >>>> state
> >>>>>>>>>>> both
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreams and StreamThread.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> After further understanding of KafkaStreams
> >>>> thread
> >>>>>>>>>>> and
> >>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am currently rewriting the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Dec 29, 2017 at 4:32 AM, Matthias J.
> >>>> Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Vito,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sorry for this late reply.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There can be two cases:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>  - either a store got migrated way and
> thus, is
> >>>>>>> not
> >>>>>>>>>>>>> hosted
> >>>>>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> application instance anymore,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>  - or, a store is hosted but the instance
> is in
> >>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>> rebalance
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the first case, users need to rediscover
> >>>> the
> >>>>>>>>>>> store.
> >>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case, they need to wait until rebalance is
> >>>>>>> finished.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If KafkaStreams is in state ERROR,
> >>>>>>>>>>> PENDING_SHUTDOWN, or
> >>>>>>>>>>>>>>>>>>>>>> NOT_RUNNING,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uses cannot query at all and thus they
> cannot
> >>>>>>>>>>>> rediscover
> >>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>> retry.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Does this make sense?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 12/20/17 12:54 AM, vito jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@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 <
> >>>>>>>>>>>>>>>>>>>>>> vito@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
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> --
> >>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>
> >>>>
> >>>>
> >>
> >
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Thanks Navinder and Matthias.

---
Vito


On Sat, Feb 22, 2020 at 8:12 PM Matthias J. Sax <mj...@apache.org> wrote:

> Good find Vito!
>
> What Navinder says makes sense -- as there is no RC for 2.5.0 yet, I
> took the liberty to do a HOTFIX PR so we can address the issue in 2.5.0
> already.
>
> https://github.com/apache/kafka/pull/8158
>
>
> -Matthias
>
> On 2/21/20 11:11 PM, Navinder Brar wrote:
> > Hi Vito,
> >
> > I checked the code and I think you are right. If a user provides a
> wrong partition there will be NPE at
> tasks.get(keyTaskId).getStore(storeName) as that task is not available
> at this machine.
> >
> > I think we split the line:
>
> https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L62
> into 2 parts and check tasks.get(keyTaskId) separately. If it is null,
> we can throw an InvalidPartitionException. WDYS?
> >
> > Thanks,
> > Navinder
> >
> >
> >     On Saturday, 22 February, 2020, 06:22:14 am IST, Vito Jeng
> <vi...@is-land.com.tw> wrote:
> >
> >  Hi, Matthias and Navinder,
> >
> > I have a question about the valid partition in
> > StreamThreadStateStoreProvider.
> >
> > In the StreamThreadStateStoreProvider#createKeyTaskId(storeName,
> partition):
> >
>
> https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L103
> >
> > We pass an integer as partition and then use this partition to create
> > TaskId instance in the topic group while loop. How do we make sure the
> > partition is valid? If we pass a correct storeName and a invalid
> partition
> > into createKeyTaskId() , it still looks can be created a new TaskId and
> > would not throw InvalidStateStorePartitionException.
> >
> > I guess this would cause a NullPointerException at line #62 because this
> > keyTaskId cannot found in the task list.
> >
>
> https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L62
> >
> > Does this right? or there something wrong with me?
> >
> > ---
> > Vito
> >
> >
> > On Wed, Feb 5, 2020 at 2:53 AM Navinder Brar
> > <na...@yahoo.com.invalid> wrote:
> >
> >> Thanks Vito, for incorporating this. Makes sense.
> >>
> >> -Navinder
> >>
> >>
> >> On Wednesday, February 5, 2020, 12:17 AM, Matthias J. Sax <
> >> mjsax@apache.org> wrote:
> >>
> > Thanks Vito!
> >
> > That makes sense to me.
> >
> >
> > On 2/1/20 11:29 PM, Vito Jeng wrote:
> >>>> Hi, folks,
> >>>>
> >>>> KIP-562(KAFKA-9445) already merged three days ago.
> >>>>
> >>>> I have updated KIP-216 to reflect the KIP-562. The main change is
> >>>> to introduce a new exception `InvalidStateStorePartitionException`,
> >>>> will be thrown when user requested partition not available.
> >>>>
> >>>> Please take a look and any feedback is welcome. Thanks Matthias for
> >>>> the reminder.
> >>>>
> >>>> --- Vito
> >>>>
> >>>>
> >>>> On Thu, Jan 23, 2020 at 2:13 PM Vito Jeng <vi...@is-land.com.tw>
> >>>> wrote:
> >>>>
> >>>>> Got it, thanks Matthias.
> >>>>>
> >>>>> --- Vito
> >>>>>
> >>>>>
> >>>>> On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax
> >>>>> <ma...@confluent.io> wrote:
> >>>>>
> >>>>>> Thanks Vito.
> >>>>>>
> >>>>>> I am also ok with either name. Just a personal slight
> >>>>>> preference, but not a important.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 1/21/20 6:52 PM, Vito Jeng wrote:
> >>>>>>> Thanks Matthias.
> >>>>>>>
> >>>>>>> The KIP is about InvalidStateStoreException. I pick
> >>>>>>> `StateStoreNotAvailableException` because it may be more
> >>>>>> intuitive
> >>>>>>> than `StreamsNotRunningException`.
> >>>>>>>
> >>>>>>> No matter which one picked, it's good to me.
> >>>>>>>
> >>>>>>> --- Vito
> >>>>>>>
> >>>>>>>
> >>>>>>> On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax
> >>>>>>> <ma...@confluent.io> wrote:
> >>>>>>>
> >>>>>>>> Thanks for updating the KIP!
> >>>>>>>>
> >>>>>>>> One last comment/question: you kept
> >>>>>>>> `StateStoreNotAvailableException`
> >>>>>> in
> >>>>>>>> favor of `StreamsNotRunningException` (to merge both as
> >>>>>>>> suggested).
> >>>>>>>>
> >>>>>>>> I am wondering, if it might be better to keep
> >>>>>>>> `StreamsNotRunningException` instead of
> >>>>>>>> `StateStoreNotAvailableException`, because this exception
> >>>>>>>> is thrown if Streams is in state PENDING_SHUTDOWN /
> >>>>>>>> NOT_RUNNING / ERROR ?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>> On 1/17/20 9:56 PM, John Roesler wrote:
> >>>>>>>>> Thanks, Vito. I've just cast my vote. -John
> >>>>>>>>>
> >>>>>>>>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
> >>>>>>>>>> Hi, folks,
> >>>>>>>>>>
> >>>>>>>>>> Just update the KIP, please take a look.
> >>>>>>>>>>
> >>>>>>>>>> Thanks!
> >>>>>>>>>>
> >>>>>>>>>> --- Vito
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng
> >>>>>>>>>> <vi...@is-land.com.tw>
> >>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Thanks Bill, John and Matthias. Glad you guys joined
> >>>>>>>>>>> this
> >>>>>> discussion.
> >>>>>>>>>>> I got a lot out of the discussion.
> >>>>>>>>>>>
> >>>>>>>>>>> I would like to update KIP-216 base on John's
> >>>>>>>>>>> suggestion to remove
> >>>>>> the
> >>>>>>>>>>> category.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> --- Vito
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
> >>>>>> matthias@confluent.io
> >>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>>> Nevertheless, if we omit the categorization, it’s
> >>>>>>>>>>>>> moot.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Ack.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I am fine to remove the middle tier. As John
> >>>>>>>>>>>> pointed out, it might
> >>>>>> be
> >>>>>>>>>>>> weird to have only one concrete exception type per
> >>>>>>>>>>>> category. We can
> >>>>>>>> also
> >>>>>>>>>>>> explain in detail how to handle each exception in
> >>>>>>>>>>>> their JavaDocs.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
> >>>>>>>>>>>>> Vito,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks for the updates, the KIP LGTM.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Bill
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <
> >>>>>> vvcephei@apache.org>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Vito,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Haha, your archive game is on point!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> What Matthias said in that email is essentially
> >>>>>>>>>>>>>> what I figured
> >>>>>> was
> >>>>>>>> the
> >>>>>>>>>>>>>> rationale. It makes sense, but the point I was
> >>>>>>>>>>>>>> making is that
> >>>>>> this
> >>>>>>>>>>>> really
> >>>>>>>>>>>>>> doesn’t seem like a good way to structure a
> >>>>>>>>>>>>>> production app. On
> >>>>>> the
> >>>>>>>>>>>> other
> >>>>>>>>>>>>>> hand, considering the exception fatal has a
> >>>>>>>>>>>>>> good chance of
> >>>>>> avoiding
> >>>>>>>> a
> >>>>>>>>>>>>>> frustrating debug session if you just forgot to
> >>>>>>>>>>>>>> call start.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Nevertheless, if we omit the categorization,
> >>>>>>>>>>>>>> it’s moot.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> It would be easy to add a categorization layer
> >>>>>>>>>>>>>> later if we want
> >>>>>> it,
> >>>>>>>> but
> >>>>>>>>>>>>>> not very easy to change it if we get it wrong.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for your consideration! -John
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>> Hi John,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> About `StreamsNotStartedException is strange`
> >>>>>>>>>>>>>>> -- The original idea came from Matthias, two
> >>>>>>>>>>>>>>> years ago. :) You can reference here:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6
> > c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> > <
> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> >
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>
> > About omitting the categorization --
> >>>>>>>>>>>>>>> It looks reasonable. I'm fine with omitting
> >>>>>>>>>>>>>>> the categorization
> >>>>>> but
> >>>>>>>> not
> >>>>>>>>>>>>>> very
> >>>>>>>>>>>>>>> sure it is a good choice. Does any other
> >>>>>>>>>>>>>>> folks provide opinion?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi, folks,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Just update the KIP-216, please take a look.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng
> >>>>>>>>>>>>>>> <vi...@is-land.com.tw>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi, folks,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thank you suggestion, really appreciate it.
> >>>>>>>>>>>>>>>> :) I understand your concern. I'll merge
> >>>>>> StreamsNotRunningException
> >>>>>>>> and
> >>>>>>>>>>>>>>>> StateStoreNotAvailableException.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John
> >>>>>>>>>>>>>>>> Roesler <
> >>>>>> vvcephei@apache.org
> >>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hey Vito,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Yes, thanks for the KIP. Sorry the
> >>>>>>>>>>>>>>>>> discussion has been so
> >>>>>> long.
> >>>>>>>>>>>>>>>>> Hopefully, we can close it out soon.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I agree we can drop
> >>>>>>>>>>>>>>>>> StreamsNotRunningException in favor of
> >>>>>>>>>>>>>>>>> just StateStoreNotAvailableException.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Unfortunately, I have some higher-level
> >>>>>>>>>>>>>>>>> concerns. The value of these exceptions
> >>>>>>>>>>>>>>>>> is that they tell you how to handle the
> >>>>>>>>>>>>>>>>> various situations that can arise while
> >>>>>>>>>>>>>>>>> querying a distributed data store.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Ideally, as a caller, I should be able to
> >>>>>>>>>>>>>>>>> just catch
> >>>>>> "retriable"
> >>>>>>>> or
> >>>>>>>>>>>>>>>>> "fatal" and handle them appropriately.
> >>>>>>>>>>>>>>>>> Otherwise, there's no point in having
> >>>>>>>>>>>>>>>>> categories, and we should just have all
> >>>>>>>>>>>>>>>>> the exceptions extend
> >>>>>>>>>>>>>>>>> InvalidStateStoreException.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Presently, it's not possible to tell from
> >>>>>>>>>>>>>>>>> just the "retriable"/"fatal" distinction
> >>>>>>>>>>>>>>>>> what to do. You  can tell from the
> >>>>>>>>>>>>>>>>> descriptions of the various exceptions.
> >>>>>>>>>>>>>>>>> E.g.:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Retriable: * StreamsRebalancingException:
> >>>>>>>>>>>>>>>>> the exact same call should just be
> >>>>>>>>>>>>>>>>> retried until the rebalance is complete *
> >>>>>>>>>>>>>>>>> StateStoreMigratedException: the store
> >>>>>>>>>>>>>>>>> handle is now invalid, so you need to
> >>>>>>>>>>>>>>>>> re-discover the instance and get a new
> >>>>>>>>>>>>>>>>> handle on that instance. In other words,
> >>>>>>>>>>>>>>>>> the query itself may be valid, but the
> >>>>>>>>>>>>>>>>> particular method invocation on this
> >>>>>>>>>>>>>>>>> particular instance has encountered a
> >>>>>>>>>>>>>>>>> fatal exception.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Fatal: * UnknownStateStoreException: this
> >>>>>>>>>>>>>>>>> is truly fatal. No amount of retrying or
> >>>>>>>>>>>>>>>>> re-discovering is going to get you a
> >>>>>>>>>>>>>>>>> handle
> >>>>>>>> on a
> >>>>>>>>>>>>>>>>> store that doesn't exist in the cluster.
> >>>>>>>>>>>>>>>>> * StateStoreNotAvailableException: this
> >>>>>>>>>>>>>>>>> is actually
> >>>>>> recoverable,
> >>>>>>>>>>>>>>>>> since the store might exist in the
> >>>>>>>>>>>>>>>>> cluster, but isn't
> >>>>>>>> available
> >>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>> this particular instance (which is shut
> >>>>>>>>>>>>>>>>> down or whatever).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Personally, I'm not a fan of code
> >>>>>>>>>>>>>>>>> bureaucracy, so I'm 100%
> >>>>>> fine
> >>>>>>>>>>>>>>>>> with omitting the categorization and just
> >>>>>>>>>>>>>>>>> having 5 subclasses of
> >>>>>>>>>>>>>>>>> InvalidStateStoreException. Each of them
> >>>>>>>>>>>>>>>>> would tell you how to handle them, and
> >>>>>>>>>>>>>>>>> it's not too many to really understand
> >>>>>>>>>>>>>>>>> and handle each one.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> If you really want to have a middle tier,
> >>>>>>>>>>>>>>>>> I'd recommend: *
> >>>>>>>>>>>>>>>>> RetryableStateStoreException: the exact
> >>>>>>>>>>>>>>>>> same call should be repeated. *
> >>>>>>>>>>>>>>>>> RecoverableStateStoreException: the store
> >>>>>>>>>>>>>>>>> handle should be discarded and the caller
> >>>>>>>>>>>>>>>>> should re-discover the location of the
> >>>>>>>>>>>>>>>>> store and repeat the query on the correct
> >>>>>>>>>>>>>>>>> instance. * FatalStateStoreException: the
> >>>>>>>>>>>>>>>>> query/request is totally invalid and will
> >>>>>>>>>>>>>>>>> never succeed.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> However, attempting to categorize the
> >>>>>>>>>>>>>>>>> proposed exceptions reveals even problems
> >>>>>>>>>>>>>>>>> with this categorization: Retriable: *
> >>>>>>>>>>>>>>>>> StreamsRebalancingException Recoverable:
> >>>>>>>>>>>>>>>>> * StateStoreMigratedException *
> >>>>>>>>>>>>>>>>> StreamsNotRunningException Fatal: *
> >>>>>>>>>>>>>>>>> UnknownStateStoreException
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> But StreamsNotStartedException is
> >>>>>>>>>>>>>>>>> strange... It means that one code path
> >>>>>>>>>>>>>>>>> got a handle on a specific KafkaStreams
> >>>>>>>>>>>>>>>>> object instance and sent it a query
> >>>>>>>>>>>>>>>>> before another code path invoked the
> >>>>>>>>>>>>>>>>> start() method on the exact same object
> >>>>>>>>>>>>>>>>> instance. It seems like the most likely
> >>>>>>>>>>>>>>>>> scenario is that whoever wrote the
> >>>>>>>>>>>>>>>>> program just forgot to call start()
> >>>>>>>>>>>>>>>>> before querying, in which case, retrying
> >>>>>>>>>>>>>>>>> isn't going to help, and a fatal
> >>>>>> exception
> >>>>>>>>>>>>>>>>> is more appropriate. I.e., it sounds like
> >>>>>>>>>>>>>>>>> a "first 15 minutes experience" problem,
> >>>>>>>>>>>>>>>>> and making it fatal would be more
> >>>>>>>>>>>>>>>>> helpful. Even in a production context,
> >>>>>>>>>>>>>>>>> there's no reason not to sequence your
> >>>>>>>>>>>>>>>>> application startup such that you don't
> >>>>>>>>>>>>>>>>> accept queries until after Streams is
> >>>>>>>>>>>>>>>>> started. Thus, I guess I'd categorize it
> >>>>>>>>>>>>>>>>> under "fatal".
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regardless of whether you make it fatal
> >>>>>>>>>>>>>>>>> or retriable, you'd still have a whole
> >>>>>>>>>>>>>>>>> category with only one exception in it,
> >>>>>>>>>>>>>>>>> and the other two categories only have
> >>>>>>>>>>>>>>>>> two exceptions. Plus, as you pointed out
> >>>>>>>>>>>>>>>>> in the KIP, you can't get all exceptions
> >>>>>>>>>>>>>>>>> in all cases anyway: * store() can only
> >>>>>>>>>>>>>>>>> throw NotStarted, NotRunning, and
> >>>>>>>>>>>>>>>>> Unknown * actual store queries can only
> >>>>>>>>>>>>>>>>> throw Rebalancing, Migrated, and
> >>>>>>>>>>>>>>>>> NotRunning
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thus, in practice also, there are exactly
> >>>>>>>>>>>>>>>>> three categories and also exactly three
> >>>>>>>>>>>>>>>>> exception types. It doesn't seem like
> >>>>>>>>>>>>>>>>> there's a great advantage to the
> >>>>>>>>>>>>>>>>> categories here. To avoid the
> >>>>>>>>>>>>>>>>> categorization problem and also to
> >>>>>>>>>>>>>>>>> clarify what exceptions can actually be
> >>>>>>>>>>>>>>>>> thrown in different circumstances, it
> >>>>>>>>>>>>>>>>> seems like we should just: * get rid of
> >>>>>>>>>>>>>>>>> the middle tier and make all the
> >>>>>>>>>>>>>>>>> exceptions extend
> >>>>>>>>>>>>>>>>> InvalidStateStoreException * drop
> >>>>>>>>>>>>>>>>> StateStoreNotAvailableException in favor
> >>>>>>>>>>>>>>>>> of StreamsNotRunningException * clearly
> >>>>>>>>>>>>>>>>> document on all public methods which
> >>>>>>>>>>>>>>>>> exceptions need to be handled
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> How do you feel about this? Thanks,
> >>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill
> >>>>>>>>>>>>>>>>> Bejeck wrote:
> >>>>>>>>>>>>>>>>>> Thanks for KIP Vito.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Overall the KIP LGTM, but I'd have to
> >>>>>>>>>>>>>>>>>> agree with others on
> >>>>>>>> merging
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>>>>> classes.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Since in both cases, the thread state
> >>>>>>>>>>>>>>>>>> is in
> >>>>>> `PENDING_SHUTDOWN ||
> >>>>>>>>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure
> >>>>>>>>>>>>>>>>>> how we could
> >>>>>> distinguish
> >>>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>> use the different exceptions.  Maybe a
> >>>>>>>>>>>>>>>>>> good middle ground would be to have a
> >>>>>>>> detailed
> >>>>>>>>>>>>>>>>>> exception message.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The KIP freeze is close, so I think if
> >>>>>>>>>>>>>>>>>> we can agree on this,
> >>>>>> we
> >>>>>>>> can
> >>>>>>>>>>>>>>>>> wrap up
> >>>>>>>>>>>>>>>>>> the voting soon.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks, Bill
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM
> >>>>>>>>>>>>>>>>>> Matthias J. Sax <
> >>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Vito,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> It's still unclear to me what the
> >>>>>>>>>>>>>>>>>>> advantage is, to have both
> >>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> For both cased, the state is
> >>>>>>>>>>>>>>>>>>> `PENDING_SHUTDOWN /
> >>>>>> NOT_RUNNING /
> >>>>>>>>>>>>>> ERROR`
> >>>>>>>>>>>>>>>>>>> and thus, for a user point of view,
> >>>>>>>>>>>>>>>>>>> why does it matter if
> >>>>>> the
> >>>>>>>>>>>>>> store is
> >>>>>>>>>>>>>>>>>>> closed on not? I don't understand
> >>>>>>>>>>>>>>>>>>> why/how this information
> >>>>>>>> would
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> useful? Do you have a concrete
> >>>>>>>>>>>>>>>>>>> example in mind how a user
> >>>>>> would
> >>>>>>>>>>>>>> react
> >>>>>>>>>>>>>>>>>>> differently to both exceptions?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> @Vinoth: about
> >>>>>>>>>>>>>>>>>>> `StreamsRebalancingException` -- to
> >>>>>>>>>>>>>>>>>>> me, it
> >>>>>> seems
> >>>>>>>>>>>>>> best
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> actually do this on a per-query
> >>>>>>>>>>>>>>>>>>> basis, ie, have an overload
> >>>>>>>>>>>>>>>>>>> `KafkaStreams#store(...)` that takes
> >>>>>>>>>>>>>>>>>>> a boolean flag that
> >>>>>> allow
> >>>>>>>> to
> >>>>>>>>>>>>>>>>>>> _disable_ the exception and opt-in to
> >>>>>>>>>>>>>>>>>>> query a active store
> >>>>>>>> during
> >>>>>>>>>>>>>>>>>>> recovery. However, as KIP-535
> >>>>>>>>>>>>>>>>>>> actually introduces this
> >>>>>> change
> >>>>>>>> in
> >>>>>>>>>>>>>>>>>>> behavior, I think KIP-216 should not
> >>>>>>>>>>>>>>>>>>> cover this, but KIP-535
> >>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>> updated. I'll follow up on the other
> >>>>>>>>>>>>>>>>>>> KIP thread to raise
> >>>>>> this
> >>>>>>>>>>>>>> point.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>> Hi, Matthias & Vinoth,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> What is still unclear to me is,
> >>>>>>>>>>>>>>>>>>>>> what we gain by having
> >>>>>> both
> >>>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`. Both
> >>>>>>>>>>>>>>>>>>>>> exception are thrown when
> >>>>>>>>>>>>>>>>>>>>> KafkaStreams is in state
> >>>>>>>>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a
> >>>>>>>>>>>>>>>>>>>>> user what do I gain to
> >>>>>> know
> >>>>>>>>>>>>>> if the
> >>>>>>>>>>>>>>>>>>>>> state store is closed on not -- I
> >>>>>>>>>>>>>>>>>>>>> can't query it anyway?
> >>>>>>>> Maybe
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>> miss
> >>>>>>>>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Yes, both
> >>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>>>>>>>> are fatal exception. But
> >>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>>>>>>>> is fatal exception
> >>>>>> about
> >>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> related. I think it would be
> >>>>>>>>>>>>>>>>>>>> helpful that if user need to
> >>>>>> distinguish
> >>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>>>>> different case to handle it.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I'm not very sure, does that make
> >>>>>>>>>>>>>>>>>>>> sense?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM
> >>>>>>>>>>>>>>>>>>>> Vinoth Chandar <
> >>>>>>>>>>>>>> vinoth@apache.org>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> +1 on merging
> >>>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`,
> >>>>>>>>>>>>>>>>>>>>> both exceptions are
> >>>>>> fatal
> >>>>>>>>>>>>>>>>> anyway. IMO
> >>>>>>>>>>>>>>>>>>>>> its best to have these exceptions
> >>>>>>>>>>>>>>>>>>>>> be about the state store
> >>>>>>>>>>>>>> (and not
> >>>>>>>>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>>>>>> state), to easier understanding.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Additionally, KIP-535 allows for
> >>>>>>>>>>>>>>>>>>>>> querying of state stores
> >>>>>> in
> >>>>>>>>>>>>>>>>> rebalancing
> >>>>>>>>>>>>>>>>>>>>> state. So do we need the
> >>>>>>>>>>>>>>>>>>>>> StreamsRebalancingException?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias
> >>>>>>>>>>>>>>>>>>>>> J. Sax" <
> >>>>>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>> Sorry that I dropped the ball
> >>>>>>>>>>>>>>>>>>>>>> on this...
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP.
> >>>>>>>>>>>>>>>>>>>>>> Overall LGTM now. Feel free
> >>>>>> to
> >>>>>>>>>>>>>> start
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> VOTE
> >>>>>>>>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> What is still unclear to me is,
> >>>>>>>>>>>>>>>>>>>>>> what we gain by having
> >>>>>> both
> >>>>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`.
> >>>>>>>>>>>>>>>>>>> Both
> >>>>>>>>>>>>>>>>>>>>>> exception are thrown when
> >>>>>>>>>>>>>>>>>>>>>> KafkaStreams is in state
> >>>>>>>>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as
> >>>>>>>>>>>>>>>>>>>>>> a user what do I gain to
> >>>>>> know
> >>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> state store is closed on not --
> >>>>>>>>>>>>>>>>>>>>>> I can't query it anyway?
> >>>>>>>>>>>>>> Maybe I
> >>>>>>>>>>>>>>>>> miss
> >>>>>>>>>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply,
> >>>>>>>>>>>>>>>>>>>>>>> thanks for the review.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> > Why is it only thrown if the state is REBALANCING? A
> >>>>>> store
> >>>>>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>>>>> migrated during a
> >>>>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
> >>>>>>>>>>>>>>>>>>>>>>>> Streams might
> >>>>>>>> resume
> >>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>>>> RUNNING state and afterward
> >>>>>>>>>>>>>>>>>>>>>>>> somebody tries to use an
> >>>>>> old
> >>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>>>>>>> Also, if state is
> >>>>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
> >>>>>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
> >>>>>>>>>>>>>>>>>>>>>>>> Hence, I think
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>>>>>>>> does only make sense
> >>>>>> during
> >>>>>>>>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thank you point this, already
> >>>>>>>>>>>>>>>>>>>>>>> updated.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Why do we need to distinguish
> >>>>>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> > `KafkaStreamsNotRunningException` may be caused by
> >>>>>> various
> >>>>>>>>>>>>>>>>> reasons, I
> >>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>> it would be helpful that the
> >>>>>>>>>>>>>>>>>>>>>>> user can distinguish whether
> >>>>>>>>>>>>>>>>>>>>>>> it is caused by the state
> >>>>>>>> store
> >>>>>>>>>>>>>>>>> closed.
> >>>>>>>>>>>>>>>>>>>>>>> (Maybe I am wrong...)
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Last, why do we distinguish
> >>>>>>>>>>>>>>>>>>>>>>> between `KafkaStreams`
> >>>>>> instance
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it
> >>>>>>>>>>>>>>>>>>>>>>>> seems we should always
> >>>>>> refer to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>>>>>>>> because that is the level
> >>>>>>>>>>>>>>>>>>>>>>>> of granularity in which we
> >>>>>>>>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>>>>>>>> IQ atm.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Totally agree. Do you mean
> >>>>>>>>>>>>>>>>>>>>>>> the naming of state store
> >>>>>>>>>>>>>> exceptions?
> >>>>>>>>>>>>>>>>>>>>>>> I don't have special reason
> >>>>>>>>>>>>>>>>>>>>>>> to distinguish these two.
> >>>>>>>>>>>>>>>>>>>>>>> Your suggestion look more
> >>>>>>>>>>>>>>>>>>>>>>> reasonable for the exception
> >>>>>>>>>>>>>> naming.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Last, for
> >>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
> >>>>>>>>>>>>>>>>>>>>>>> I would add
> >>>>>> that a
> >>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> rediscover the store and
> >>>>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>> store
> >>>>>>>>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>>>>>>>> invalid and a new store
> >>>>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
> >>>>>>>>>>>>>>>>>>>>>>>> That
> >>>>>> is
> >>>>>>>> a
> >>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
> >>>>>>>>>>>>>>>>>>>>>>>> that allows for
> >>>>>>>>>>>>>> "blind"
> >>>>>>>>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>>>>>>>> that either resolve (if the
> >>>>>>>>>>>>>>>>>>>>>>>> store is still on the same
> >>>>>>>>>>>>>> instance
> >>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
> >>>>>>>>>>>>>>>>>>>>>>>> changes to
> >>>>>>>>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> store was migrated away
> >>>>>>>>>>>>>>>>>>>>>>>> during rebalancing).
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> The KIP already updated,
> >>>>>>>>>>>>>>>>>>>>>>> please take a look. :)
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48
> >>>>>>>>>>>>>>>>>>>>>>> PM Matthias J. Sax <
> >>>>>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Any update on this KIP?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM,
> >>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply.
> >>>>>>>>>>>>>>>>>>>>>>>>> The 2.4 deadline kept us
> >>>>>> quite
> >>>>>>>>>>>>>> busy.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> > Why is it only thrown if the state is REBALANCING? A
> >>>>>>>> store
> >>>>>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>>>>>> migrated during a
> >>>>>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
> >>>>>>>>>>>>>>>>>>>>>>>>> Streams might
> >>>>>>>> resume
> >>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>>>>> RUNNING state and
> >>>>>>>>>>>>>>>>>>>>>>>>> afterward somebody tries
> >>>>>>>>>>>>>>>>>>>>>>>>> to use an
> >>>>>> old
> >>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>>>>>>>> Also, if state is
> >>>>>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
> >>>>>>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
> >>>>>>>>>>>>>>>>>>>>>>>>> Hence, I think
> >>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>>>>>>>>> does only make sense
> >>>>>> during
> >>>>>>>>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Why do we need to
> >>>>>>>>>>>>>>>>>>>>>>>>> distinguish between
> >>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> > and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Last, why do we
> >>>>>>>>>>>>>>>>>>>>>>>>> distinguish between
> >>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreams`
> >>>>>>>>>>>>>> instance and
> >>>>>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me,
> >>>>>>>>>>>>>>>>>>>>>>>>> it seems we should
> >>>>>>>>>>>>>>>>>>>>>>>>> always
> >>>>>> refer
> >>>>>>>> to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>>>>>>>>> because that is the level
> >>>>>>>>>>>>>>>>>>>>>>>>> of granularity in which
> >>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>>>>>>>> IQ
> >>>>>>>>>>>>>>>>>>>>>>>> atm.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Last, for
> >>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
> >>>>>>>>>>>>>>>>>>>>>>>>> I would add
> >>>>>>>> that a
> >>>>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>> rediscover the store and
> >>>>>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
> >>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>> store
> >>>>>>>>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>>>>>>>>> invalid and a new store
> >>>>>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
> >>>>>>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
> >>>>>>>>>>>>>>>>>>>>>>>>> that allows for
> >>>>>>>>>>>>>> "blind"
> >>>>>>>>>>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>>>>>>>>> that either resolve (if
> >>>>>>>>>>>>>>>>>>>>>>>>> the store is still on the
> >>>>>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>> instance
> >>>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
> >>>>>>>>>>>>>>>>>>>>>>>>> changes to
> >>>>>>>>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> store was migrated away
> >>>>>>>>>>>>>>>>>>>>>>>>> during rebalancing).
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito
> >>>>>>>>>>>>>>>>>>>>>>>>> Jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>> My bad. The short link
> >>>>>>>>>>>>>>>>>>>>>>>>>> `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> > <https://shorturl.at/CDNT9>
> >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> > <https://shorturl.at/CDNT9>
> >>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> > <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Please use the
> >>>>>>>>>>>>>>>>>>>>>>>>>> following instead:
> >>>>>>>>>>>>>> https://shorturl.at/bkKQU
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at
> >>>>>>>>>>>>>>>>>>>>>>>>>> 10:53 AM Vito Jeng <
> >>>>>>>>>>>>>>>>> vito@is-land.com.tw>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > Thank you for explanation. I agree with your
> >>>>>> opinion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>> would never throw
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > For the case that corresponding thread crashes
> >>>>>> after we
> >>>>>>>>>>>>>>>>> handed out
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> store handle. We may
> >>>>>>>>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>>>>> when stream
> >>>>>> thread is
> >>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> running(
> >>>>>>>>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> or throw
> >>>>>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>> store is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> closed(https://shorturl.at/hrvAN).
> >>>>>>>>>>>>>>>>>>>>>>>>>>> So I
> >>>>>> think
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> add a new type for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> this case. Does that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> vs
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > I understand your point. I rename
> >>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > About check unknown state store names:
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> hint. I add a new
> >>>>>>>>>>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>>>> `UnknownStateStoreException`
> >>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> this case.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, we should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> still have fatal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> exception
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure why you
> >>>>>>>>>>>>>> remove
> >>>>>>>>>>>>>>>>> it?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you point this,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> already add it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> again.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP already
> >>>>>>>>>>>>>>>>>>>>>>>>>>> updated, please take
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a look.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>
> >>
> >>
> >>
> >>
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <mj...@apache.org>.
Good find Vito!

What Navinder says makes sense -- as there is no RC for 2.5.0 yet, I
took the liberty to do a HOTFIX PR so we can address the issue in 2.5.0
already.

https://github.com/apache/kafka/pull/8158


-Matthias

On 2/21/20 11:11 PM, Navinder Brar wrote:
> Hi Vito,
>
> I checked the code and I think you are right. If a user provides a
wrong partition there will be NPE at
tasks.get(keyTaskId).getStore(storeName) as that task is not available
at this machine.
>
> I think we split the line:
https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L62
into 2 parts and check tasks.get(keyTaskId) separately. If it is null,
we can throw an InvalidPartitionException. WDYS?
>
> Thanks,
> Navinder
>
>
>     On Saturday, 22 February, 2020, 06:22:14 am IST, Vito Jeng
<vi...@is-land.com.tw> wrote:
>
>  Hi, Matthias and Navinder,
>
> I have a question about the valid partition in
> StreamThreadStateStoreProvider.
>
> In the StreamThreadStateStoreProvider#createKeyTaskId(storeName,
partition):
>
https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L103
>
> We pass an integer as partition and then use this partition to create
> TaskId instance in the topic group while loop. How do we make sure the
> partition is valid? If we pass a correct storeName and a invalid partition
> into createKeyTaskId() , it still looks can be created a new TaskId and
> would not throw InvalidStateStorePartitionException.
>
> I guess this would cause a NullPointerException at line #62 because this
> keyTaskId cannot found in the task list.
>
https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L62
>
> Does this right? or there something wrong with me?
>
> ---
> Vito
>
>
> On Wed, Feb 5, 2020 at 2:53 AM Navinder Brar
> <na...@yahoo.com.invalid> wrote:
>
>> Thanks Vito, for incorporating this. Makes sense.
>>
>> -Navinder
>>
>>
>> On Wednesday, February 5, 2020, 12:17 AM, Matthias J. Sax <
>> mjsax@apache.org> wrote:
>>
> Thanks Vito!
> 
> That makes sense to me.
> 
> 
> On 2/1/20 11:29 PM, Vito Jeng wrote:
>>>> Hi, folks,
>>>>
>>>> KIP-562(KAFKA-9445) already merged three days ago.
>>>>
>>>> I have updated KIP-216 to reflect the KIP-562. The main change is
>>>> to introduce a new exception `InvalidStateStorePartitionException`,
>>>> will be thrown when user requested partition not available.
>>>>
>>>> Please take a look and any feedback is welcome. Thanks Matthias for
>>>> the reminder.
>>>>
>>>> --- Vito
>>>>
>>>>
>>>> On Thu, Jan 23, 2020 at 2:13 PM Vito Jeng <vi...@is-land.com.tw>
>>>> wrote:
>>>>
>>>>> Got it, thanks Matthias.
>>>>>
>>>>> --- Vito
>>>>>
>>>>>
>>>>> On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax
>>>>> <ma...@confluent.io> wrote:
>>>>>
>>>>>> Thanks Vito.
>>>>>>
>>>>>> I am also ok with either name. Just a personal slight
>>>>>> preference, but not a important.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 1/21/20 6:52 PM, Vito Jeng wrote:
>>>>>>> Thanks Matthias.
>>>>>>>
>>>>>>> The KIP is about InvalidStateStoreException. I pick
>>>>>>> `StateStoreNotAvailableException` because it may be more
>>>>>> intuitive
>>>>>>> than `StreamsNotRunningException`.
>>>>>>>
>>>>>>> No matter which one picked, it's good to me.
>>>>>>>
>>>>>>> --- Vito
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax
>>>>>>> <ma...@confluent.io> wrote:
>>>>>>>
>>>>>>>> Thanks for updating the KIP!
>>>>>>>>
>>>>>>>> One last comment/question: you kept
>>>>>>>> `StateStoreNotAvailableException`
>>>>>> in
>>>>>>>> favor of `StreamsNotRunningException` (to merge both as
>>>>>>>> suggested).
>>>>>>>>
>>>>>>>> I am wondering, if it might be better to keep
>>>>>>>> `StreamsNotRunningException` instead of
>>>>>>>> `StateStoreNotAvailableException`, because this exception
>>>>>>>> is thrown if Streams is in state PENDING_SHUTDOWN /
>>>>>>>> NOT_RUNNING / ERROR ?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 1/17/20 9:56 PM, John Roesler wrote:
>>>>>>>>> Thanks, Vito. I've just cast my vote. -John
>>>>>>>>>
>>>>>>>>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
>>>>>>>>>> Hi, folks,
>>>>>>>>>>
>>>>>>>>>> Just update the KIP, please take a look.
>>>>>>>>>>
>>>>>>>>>> Thanks!
>>>>>>>>>>
>>>>>>>>>> --- Vito
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng
>>>>>>>>>> <vi...@is-land.com.tw>
>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks Bill, John and Matthias. Glad you guys joined
>>>>>>>>>>> this
>>>>>> discussion.
>>>>>>>>>>> I got a lot out of the discussion.
>>>>>>>>>>>
>>>>>>>>>>> I would like to update KIP-216 base on John's
>>>>>>>>>>> suggestion to remove
>>>>>> the
>>>>>>>>>>> category.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --- Vito
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
>>>>>> matthias@confluent.io
>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>> Nevertheless, if we omit the categorization, it’s
>>>>>>>>>>>>> moot.
>>>>>>>>>>>>
>>>>>>>>>>>> Ack.
>>>>>>>>>>>>
>>>>>>>>>>>> I am fine to remove the middle tier. As John
>>>>>>>>>>>> pointed out, it might
>>>>>> be
>>>>>>>>>>>> weird to have only one concrete exception type per
>>>>>>>>>>>> category. We can
>>>>>>>> also
>>>>>>>>>>>> explain in detail how to handle each exception in
>>>>>>>>>>>> their JavaDocs.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
>>>>>>>>>>>>> Vito,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for the updates, the KIP LGTM.
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Bill
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <
>>>>>> vvcephei@apache.org>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Vito,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Haha, your archive game is on point!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What Matthias said in that email is essentially
>>>>>>>>>>>>>> what I figured
>>>>>> was
>>>>>>>> the
>>>>>>>>>>>>>> rationale. It makes sense, but the point I was
>>>>>>>>>>>>>> making is that
>>>>>> this
>>>>>>>>>>>> really
>>>>>>>>>>>>>> doesn’t seem like a good way to structure a
>>>>>>>>>>>>>> production app. On
>>>>>> the
>>>>>>>>>>>> other
>>>>>>>>>>>>>> hand, considering the exception fatal has a
>>>>>>>>>>>>>> good chance of
>>>>>> avoiding
>>>>>>>> a
>>>>>>>>>>>>>> frustrating debug session if you just forgot to
>>>>>>>>>>>>>> call start.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Nevertheless, if we omit the categorization,
>>>>>>>>>>>>>> it’s moot.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It would be easy to add a categorization layer
>>>>>>>>>>>>>> later if we want
>>>>>> it,
>>>>>>>> but
>>>>>>>>>>>>>> not very easy to change it if we get it wrong.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for your consideration! -John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> About `StreamsNotStartedException is strange`
>>>>>>>>>>>>>>> -- The original idea came from Matthias, two
>>>>>>>>>>>>>>> years ago. :) You can reference here:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>
>>>>>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6
> c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> <https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>
> About omitting the categorization --
>>>>>>>>>>>>>>> It looks reasonable. I'm fine with omitting
>>>>>>>>>>>>>>> the categorization
>>>>>> but
>>>>>>>> not
>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>> sure it is a good choice. Does any other
>>>>>>>>>>>>>>> folks provide opinion?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi, folks,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Just update the KIP-216, please take a look.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng
>>>>>>>>>>>>>>> <vi...@is-land.com.tw>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi, folks,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thank you suggestion, really appreciate it.
>>>>>>>>>>>>>>>> :) I understand your concern. I'll merge
>>>>>> StreamsNotRunningException
>>>>>>>> and
>>>>>>>>>>>>>>>> StateStoreNotAvailableException.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John
>>>>>>>>>>>>>>>> Roesler <
>>>>>> vvcephei@apache.org
>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hey Vito,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Yes, thanks for the KIP. Sorry the
>>>>>>>>>>>>>>>>> discussion has been so
>>>>>> long.
>>>>>>>>>>>>>>>>> Hopefully, we can close it out soon.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I agree we can drop
>>>>>>>>>>>>>>>>> StreamsNotRunningException in favor of
>>>>>>>>>>>>>>>>> just StateStoreNotAvailableException.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Unfortunately, I have some higher-level
>>>>>>>>>>>>>>>>> concerns. The value of these exceptions
>>>>>>>>>>>>>>>>> is that they tell you how to handle the
>>>>>>>>>>>>>>>>> various situations that can arise while
>>>>>>>>>>>>>>>>> querying a distributed data store.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ideally, as a caller, I should be able to
>>>>>>>>>>>>>>>>> just catch
>>>>>> "retriable"
>>>>>>>> or
>>>>>>>>>>>>>>>>> "fatal" and handle them appropriately.
>>>>>>>>>>>>>>>>> Otherwise, there's no point in having
>>>>>>>>>>>>>>>>> categories, and we should just have all
>>>>>>>>>>>>>>>>> the exceptions extend
>>>>>>>>>>>>>>>>> InvalidStateStoreException.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Presently, it's not possible to tell from
>>>>>>>>>>>>>>>>> just the "retriable"/"fatal" distinction
>>>>>>>>>>>>>>>>> what to do. You  can tell from the
>>>>>>>>>>>>>>>>> descriptions of the various exceptions.
>>>>>>>>>>>>>>>>> E.g.:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Retriable: * StreamsRebalancingException:
>>>>>>>>>>>>>>>>> the exact same call should just be
>>>>>>>>>>>>>>>>> retried until the rebalance is complete *
>>>>>>>>>>>>>>>>> StateStoreMigratedException: the store
>>>>>>>>>>>>>>>>> handle is now invalid, so you need to
>>>>>>>>>>>>>>>>> re-discover the instance and get a new
>>>>>>>>>>>>>>>>> handle on that instance. In other words,
>>>>>>>>>>>>>>>>> the query itself may be valid, but the
>>>>>>>>>>>>>>>>> particular method invocation on this
>>>>>>>>>>>>>>>>> particular instance has encountered a
>>>>>>>>>>>>>>>>> fatal exception.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Fatal: * UnknownStateStoreException: this
>>>>>>>>>>>>>>>>> is truly fatal. No amount of retrying or
>>>>>>>>>>>>>>>>> re-discovering is going to get you a
>>>>>>>>>>>>>>>>> handle
>>>>>>>> on a
>>>>>>>>>>>>>>>>> store that doesn't exist in the cluster.
>>>>>>>>>>>>>>>>> * StateStoreNotAvailableException: this
>>>>>>>>>>>>>>>>> is actually
>>>>>> recoverable,
>>>>>>>>>>>>>>>>> since the store might exist in the
>>>>>>>>>>>>>>>>> cluster, but isn't
>>>>>>>> available
>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>> this particular instance (which is shut
>>>>>>>>>>>>>>>>> down or whatever).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Personally, I'm not a fan of code
>>>>>>>>>>>>>>>>> bureaucracy, so I'm 100%
>>>>>> fine
>>>>>>>>>>>>>>>>> with omitting the categorization and just
>>>>>>>>>>>>>>>>> having 5 subclasses of
>>>>>>>>>>>>>>>>> InvalidStateStoreException. Each of them
>>>>>>>>>>>>>>>>> would tell you how to handle them, and
>>>>>>>>>>>>>>>>> it's not too many to really understand
>>>>>>>>>>>>>>>>> and handle each one.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If you really want to have a middle tier,
>>>>>>>>>>>>>>>>> I'd recommend: *
>>>>>>>>>>>>>>>>> RetryableStateStoreException: the exact
>>>>>>>>>>>>>>>>> same call should be repeated. *
>>>>>>>>>>>>>>>>> RecoverableStateStoreException: the store
>>>>>>>>>>>>>>>>> handle should be discarded and the caller
>>>>>>>>>>>>>>>>> should re-discover the location of the
>>>>>>>>>>>>>>>>> store and repeat the query on the correct
>>>>>>>>>>>>>>>>> instance. * FatalStateStoreException: the
>>>>>>>>>>>>>>>>> query/request is totally invalid and will
>>>>>>>>>>>>>>>>> never succeed.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> However, attempting to categorize the
>>>>>>>>>>>>>>>>> proposed exceptions reveals even problems
>>>>>>>>>>>>>>>>> with this categorization: Retriable: *
>>>>>>>>>>>>>>>>> StreamsRebalancingException Recoverable:
>>>>>>>>>>>>>>>>> * StateStoreMigratedException *
>>>>>>>>>>>>>>>>> StreamsNotRunningException Fatal: *
>>>>>>>>>>>>>>>>> UnknownStateStoreException
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> But StreamsNotStartedException is
>>>>>>>>>>>>>>>>> strange... It means that one code path
>>>>>>>>>>>>>>>>> got a handle on a specific KafkaStreams
>>>>>>>>>>>>>>>>> object instance and sent it a query
>>>>>>>>>>>>>>>>> before another code path invoked the
>>>>>>>>>>>>>>>>> start() method on the exact same object
>>>>>>>>>>>>>>>>> instance. It seems like the most likely
>>>>>>>>>>>>>>>>> scenario is that whoever wrote the
>>>>>>>>>>>>>>>>> program just forgot to call start()
>>>>>>>>>>>>>>>>> before querying, in which case, retrying
>>>>>>>>>>>>>>>>> isn't going to help, and a fatal
>>>>>> exception
>>>>>>>>>>>>>>>>> is more appropriate. I.e., it sounds like
>>>>>>>>>>>>>>>>> a "first 15 minutes experience" problem,
>>>>>>>>>>>>>>>>> and making it fatal would be more
>>>>>>>>>>>>>>>>> helpful. Even in a production context,
>>>>>>>>>>>>>>>>> there's no reason not to sequence your
>>>>>>>>>>>>>>>>> application startup such that you don't
>>>>>>>>>>>>>>>>> accept queries until after Streams is
>>>>>>>>>>>>>>>>> started. Thus, I guess I'd categorize it
>>>>>>>>>>>>>>>>> under "fatal".
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regardless of whether you make it fatal
>>>>>>>>>>>>>>>>> or retriable, you'd still have a whole
>>>>>>>>>>>>>>>>> category with only one exception in it,
>>>>>>>>>>>>>>>>> and the other two categories only have
>>>>>>>>>>>>>>>>> two exceptions. Plus, as you pointed out
>>>>>>>>>>>>>>>>> in the KIP, you can't get all exceptions
>>>>>>>>>>>>>>>>> in all cases anyway: * store() can only
>>>>>>>>>>>>>>>>> throw NotStarted, NotRunning, and
>>>>>>>>>>>>>>>>> Unknown * actual store queries can only
>>>>>>>>>>>>>>>>> throw Rebalancing, Migrated, and
>>>>>>>>>>>>>>>>> NotRunning
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thus, in practice also, there are exactly
>>>>>>>>>>>>>>>>> three categories and also exactly three
>>>>>>>>>>>>>>>>> exception types. It doesn't seem like
>>>>>>>>>>>>>>>>> there's a great advantage to the
>>>>>>>>>>>>>>>>> categories here. To avoid the
>>>>>>>>>>>>>>>>> categorization problem and also to
>>>>>>>>>>>>>>>>> clarify what exceptions can actually be
>>>>>>>>>>>>>>>>> thrown in different circumstances, it
>>>>>>>>>>>>>>>>> seems like we should just: * get rid of
>>>>>>>>>>>>>>>>> the middle tier and make all the
>>>>>>>>>>>>>>>>> exceptions extend
>>>>>>>>>>>>>>>>> InvalidStateStoreException * drop
>>>>>>>>>>>>>>>>> StateStoreNotAvailableException in favor
>>>>>>>>>>>>>>>>> of StreamsNotRunningException * clearly
>>>>>>>>>>>>>>>>> document on all public methods which
>>>>>>>>>>>>>>>>> exceptions need to be handled
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> How do you feel about this? Thanks,
>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill
>>>>>>>>>>>>>>>>> Bejeck wrote:
>>>>>>>>>>>>>>>>>> Thanks for KIP Vito.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Overall the KIP LGTM, but I'd have to
>>>>>>>>>>>>>>>>>> agree with others on
>>>>>>>> merging
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>>>>> classes.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Since in both cases, the thread state
>>>>>>>>>>>>>>>>>> is in
>>>>>> `PENDING_SHUTDOWN ||
>>>>>>>>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure
>>>>>>>>>>>>>>>>>> how we could
>>>>>> distinguish
>>>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>> use the different exceptions.  Maybe a
>>>>>>>>>>>>>>>>>> good middle ground would be to have a
>>>>>>>> detailed
>>>>>>>>>>>>>>>>>> exception message.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The KIP freeze is close, so I think if
>>>>>>>>>>>>>>>>>> we can agree on this,
>>>>>> we
>>>>>>>> can
>>>>>>>>>>>>>>>>> wrap up
>>>>>>>>>>>>>>>>>> the voting soon.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks, Bill
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM
>>>>>>>>>>>>>>>>>> Matthias J. Sax <
>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Vito,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> It's still unclear to me what the
>>>>>>>>>>>>>>>>>>> advantage is, to have both
>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> For both cased, the state is
>>>>>>>>>>>>>>>>>>> `PENDING_SHUTDOWN /
>>>>>> NOT_RUNNING /
>>>>>>>>>>>>>> ERROR`
>>>>>>>>>>>>>>>>>>> and thus, for a user point of view,
>>>>>>>>>>>>>>>>>>> why does it matter if
>>>>>> the
>>>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>> closed on not? I don't understand
>>>>>>>>>>>>>>>>>>> why/how this information
>>>>>>>> would
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> useful? Do you have a concrete
>>>>>>>>>>>>>>>>>>> example in mind how a user
>>>>>> would
>>>>>>>>>>>>>> react
>>>>>>>>>>>>>>>>>>> differently to both exceptions?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> @Vinoth: about
>>>>>>>>>>>>>>>>>>> `StreamsRebalancingException` -- to
>>>>>>>>>>>>>>>>>>> me, it
>>>>>> seems
>>>>>>>>>>>>>> best
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> actually do this on a per-query
>>>>>>>>>>>>>>>>>>> basis, ie, have an overload
>>>>>>>>>>>>>>>>>>> `KafkaStreams#store(...)` that takes
>>>>>>>>>>>>>>>>>>> a boolean flag that
>>>>>> allow
>>>>>>>> to
>>>>>>>>>>>>>>>>>>> _disable_ the exception and opt-in to
>>>>>>>>>>>>>>>>>>> query a active store
>>>>>>>> during
>>>>>>>>>>>>>>>>>>> recovery. However, as KIP-535
>>>>>>>>>>>>>>>>>>> actually introduces this
>>>>>> change
>>>>>>>> in
>>>>>>>>>>>>>>>>>>> behavior, I think KIP-216 should not
>>>>>>>>>>>>>>>>>>> cover this, but KIP-535
>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>> updated. I'll follow up on the other
>>>>>>>>>>>>>>>>>>> KIP thread to raise
>>>>>> this
>>>>>>>>>>>>>> point.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> Hi, Matthias & Vinoth,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> What is still unclear to me is,
>>>>>>>>>>>>>>>>>>>>> what we gain by having
>>>>>> both
>>>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`. Both
>>>>>>>>>>>>>>>>>>>>> exception are thrown when
>>>>>>>>>>>>>>>>>>>>> KafkaStreams is in state
>>>>>>>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a
>>>>>>>>>>>>>>>>>>>>> user what do I gain to
>>>>>> know
>>>>>>>>>>>>>> if the
>>>>>>>>>>>>>>>>>>>>> state store is closed on not -- I
>>>>>>>>>>>>>>>>>>>>> can't query it anyway?
>>>>>>>> Maybe
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> miss
>>>>>>>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Yes, both
>>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>>>>>>>> are fatal exception. But
>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>>>>>>>> is fatal exception
>>>>>> about
>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> related. I think it would be
>>>>>>>>>>>>>>>>>>>> helpful that if user need to
>>>>>> distinguish
>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>> different case to handle it.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I'm not very sure, does that make
>>>>>>>>>>>>>>>>>>>> sense?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM
>>>>>>>>>>>>>>>>>>>> Vinoth Chandar <
>>>>>>>>>>>>>> vinoth@apache.org>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> +1 on merging
>>>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`,
>>>>>>>>>>>>>>>>>>>>> both exceptions are
>>>>>> fatal
>>>>>>>>>>>>>>>>> anyway. IMO
>>>>>>>>>>>>>>>>>>>>> its best to have these exceptions
>>>>>>>>>>>>>>>>>>>>> be about the state store
>>>>>>>>>>>>>> (and not
>>>>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>> state), to easier understanding.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Additionally, KIP-535 allows for
>>>>>>>>>>>>>>>>>>>>> querying of state stores
>>>>>> in
>>>>>>>>>>>>>>>>> rebalancing
>>>>>>>>>>>>>>>>>>>>> state. So do we need the
>>>>>>>>>>>>>>>>>>>>> StreamsRebalancingException?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias
>>>>>>>>>>>>>>>>>>>>> J. Sax" <
>>>>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> Sorry that I dropped the ball
>>>>>>>>>>>>>>>>>>>>>> on this...
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP.
>>>>>>>>>>>>>>>>>>>>>> Overall LGTM now. Feel free
>>>>>> to
>>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> VOTE
>>>>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> What is still unclear to me is,
>>>>>>>>>>>>>>>>>>>>>> what we gain by having
>>>>>> both
>>>>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`.
>>>>>>>>>>>>>>>>>>> Both
>>>>>>>>>>>>>>>>>>>>>> exception are thrown when
>>>>>>>>>>>>>>>>>>>>>> KafkaStreams is in state
>>>>>>>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as
>>>>>>>>>>>>>>>>>>>>>> a user what do I gain to
>>>>>> know
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> state store is closed on not --
>>>>>>>>>>>>>>>>>>>>>> I can't query it anyway?
>>>>>>>>>>>>>> Maybe I
>>>>>>>>>>>>>>>>> miss
>>>>>>>>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply,
>>>>>>>>>>>>>>>>>>>>>>> thanks for the review.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
> Why is it only thrown if the state is REBALANCING? A
>>>>>> store
>>>>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>>>>> migrated during a
>>>>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
>>>>>>>>>>>>>>>>>>>>>>>> Streams might
>>>>>>>> resume
>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>> RUNNING state and afterward
>>>>>>>>>>>>>>>>>>>>>>>> somebody tries to use an
>>>>>> old
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>>>>>>> Also, if state is
>>>>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
>>>>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
>>>>>>>>>>>>>>>>>>>>>>>> Hence, I think
>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>>>>>>>> does only make sense
>>>>>> during
>>>>>>>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thank you point this, already
>>>>>>>>>>>>>>>>>>>>>>> updated.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Why do we need to distinguish
>>>>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
> `KafkaStreamsNotRunningException` may be caused by
>>>>>> various
>>>>>>>>>>>>>>>>> reasons, I
>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>> it would be helpful that the
>>>>>>>>>>>>>>>>>>>>>>> user can distinguish whether
>>>>>>>>>>>>>>>>>>>>>>> it is caused by the state
>>>>>>>> store
>>>>>>>>>>>>>>>>> closed.
>>>>>>>>>>>>>>>>>>>>>>> (Maybe I am wrong...)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Last, why do we distinguish
>>>>>>>>>>>>>>>>>>>>>>> between `KafkaStreams`
>>>>>> instance
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it
>>>>>>>>>>>>>>>>>>>>>>>> seems we should always
>>>>>> refer to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>>>>>>>> because that is the level
>>>>>>>>>>>>>>>>>>>>>>>> of granularity in which we
>>>>>>>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>>>>>>>> IQ atm.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Totally agree. Do you mean
>>>>>>>>>>>>>>>>>>>>>>> the naming of state store
>>>>>>>>>>>>>> exceptions?
>>>>>>>>>>>>>>>>>>>>>>> I don't have special reason
>>>>>>>>>>>>>>>>>>>>>>> to distinguish these two.
>>>>>>>>>>>>>>>>>>>>>>> Your suggestion look more
>>>>>>>>>>>>>>>>>>>>>>> reasonable for the exception
>>>>>>>>>>>>>> naming.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Last, for
>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
>>>>>>>>>>>>>>>>>>>>>>> I would add
>>>>>> that a
>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> rediscover the store and
>>>>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>> store
>>>>>>>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>>>>>>>> invalid and a new store
>>>>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>> is
>>>>>>>> a
>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
>>>>>>>>>>>>>>>>>>>>>>>> that allows for
>>>>>>>>>>>>>> "blind"
>>>>>>>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>>>>>>>> that either resolve (if the
>>>>>>>>>>>>>>>>>>>>>>>> store is still on the same
>>>>>>>>>>>>>> instance
>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
>>>>>>>>>>>>>>>>>>>>>>>> changes to
>>>>>>>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> store was migrated away
>>>>>>>>>>>>>>>>>>>>>>>> during rebalancing).
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> The KIP already updated,
>>>>>>>>>>>>>>>>>>>>>>> please take a look. :)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48
>>>>>>>>>>>>>>>>>>>>>>> PM Matthias J. Sax <
>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Any update on this KIP?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM,
>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply.
>>>>>>>>>>>>>>>>>>>>>>>>> The 2.4 deadline kept us
>>>>>> quite
>>>>>>>>>>>>>> busy.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
> Why is it only thrown if the state is REBALANCING? A
>>>>>>>> store
>>>>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>>>>>> migrated during a
>>>>>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
>>>>>>>>>>>>>>>>>>>>>>>>> Streams might
>>>>>>>> resume
>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>> RUNNING state and
>>>>>>>>>>>>>>>>>>>>>>>>> afterward somebody tries
>>>>>>>>>>>>>>>>>>>>>>>>> to use an
>>>>>> old
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>>>>>>>> Also, if state is
>>>>>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
>>>>>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I think
>>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>>>>>>>>> does only make sense
>>>>>> during
>>>>>>>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Why do we need to
>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between
>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
> and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Last, why do we
>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between
>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreams`
>>>>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me,
>>>>>>>>>>>>>>>>>>>>>>>>> it seems we should
>>>>>>>>>>>>>>>>>>>>>>>>> always
>>>>>> refer
>>>>>>>> to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>>>>>>>>> because that is the level
>>>>>>>>>>>>>>>>>>>>>>>>> of granularity in which
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>>>>>>>> IQ
>>>>>>>>>>>>>>>>>>>>>>>> atm.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Last, for
>>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
>>>>>>>>>>>>>>>>>>>>>>>>> I would add
>>>>>>>> that a
>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>> rediscover the store and
>>>>>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>> store
>>>>>>>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>>>>>>>>> invalid and a new store
>>>>>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
>>>>>>>>>>>>>>>>>>>>>>>>> that allows for
>>>>>>>>>>>>>> "blind"
>>>>>>>>>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>>>>>>>>> that either resolve (if
>>>>>>>>>>>>>>>>>>>>>>>>> the store is still on the
>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>> instance
>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
>>>>>>>>>>>>>>>>>>>>>>>>> changes to
>>>>>>>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> store was migrated away
>>>>>>>>>>>>>>>>>>>>>>>>> during rebalancing).
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito
>>>>>>>>>>>>>>>>>>>>>>>>> Jeng wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> My bad. The short link
>>>>>>>>>>>>>>>>>>>>>>>>>> `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
> <https://shorturl.at/CDNT9>
>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
> <https://shorturl.at/CDNT9> seems incorrect.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Please use the
>>>>>>>>>>>>>>>>>>>>>>>>>> following instead:
>>>>>>>>>>>>>> https://shorturl.at/bkKQU
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>> 10:53 AM Vito Jeng <
>>>>>>>>>>>>>>>>> vito@is-land.com.tw>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> Thank you for explanation. I agree with your
>>>>>> opinion.
>>>>>>>>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()`
>>>>>>>>>>>>>>>>>>>>>>>>>>> would never throw
>>>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
> For the case that corresponding thread crashes
>>>>>> after we
>>>>>>>>>>>>>>>>> handed out
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> store handle. We may
>>>>>>>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
> In `StreamThreadStateStoreProvider`, we would throw
>>>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>>>>> when stream
>>>>>> thread is
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> running(
>>>>>>>>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9)
>>>>>>>>>>>>>>>>>>>>>>>>>>> or throw
>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>>>>>>>>>> closed(https://shorturl.at/hrvAN).
>>>>>>>>>>>>>>>>>>>>>>>>>>> So I
>>>>>> think
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> add a new type for
>>>>>>>>>>>>>>>>>>>>>>>>>>> this case. Does that
>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> vs
>>>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
> I understand your point. I rename
>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
> About check unknown state store names:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
>>>>>>>>>>>>>>>>>>>>>>>>>>> hint. I add a new
>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>> `UnknownStateStoreException`
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> this case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>> still have fatal
>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception
>>>>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>>>>>>> Not sure why you
>>>>>>>>>>>>>> remove
>>>>>>>>>>>>>>>>> it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you point this,
>>>>>>>>>>>>>>>>>>>>>>>>>>> already add it
>>>>>>>>>>>>>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> The KIP already
>>>>>>>>>>>>>>>>>>>>>>>>>>> updated, please take
>>>>>>>>>>>>>>>>>>>>>>>>>>> a look.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>
>>
>>
>>
>>
>>


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Navinder Brar <na...@yahoo.com.INVALID>.
Hi Vito,

I checked the code and I think you are right. If a user provides a wrong partition there will be NPE at tasks.get(keyTaskId).getStore(storeName) as that task is not available at this machine.

I think we split the line: https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L62 into 2 parts and check tasks.get(keyTaskId) separately. If it is null, we can throw an InvalidPartitionException. WDYS?

Thanks,
Navinder


    On Saturday, 22 February, 2020, 06:22:14 am IST, Vito Jeng <vi...@is-land.com.tw> wrote:  
 
 Hi, Matthias and Navinder,

I have a question about the valid partition in
StreamThreadStateStoreProvider.

In the StreamThreadStateStoreProvider#createKeyTaskId(storeName, partition):
https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L103

We pass an integer as partition and then use this partition to create
TaskId instance in the topic group while loop. How do we make sure the
partition is valid? If we pass a correct storeName and a invalid partition
into createKeyTaskId() , it still looks can be created a new TaskId and
would not throw InvalidStateStorePartitionException.

I guess this would cause a NullPointerException at line #62 because this
keyTaskId cannot found in the task list.
https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L62

Does this right? or there something wrong with me?

---
Vito


On Wed, Feb 5, 2020 at 2:53 AM Navinder Brar
<na...@yahoo.com.invalid> wrote:

> Thanks Vito, for incorporating this. Makes sense.
>
> -Navinder
>
>
> On Wednesday, February 5, 2020, 12:17 AM, Matthias J. Sax <
> mjsax@apache.org> wrote:
>
> -----BEGIN PGP SIGNED MESSAGE-----
> Hash: SHA512
>
> Thanks Vito!
>
> That makes sense to me.
>
>
> On 2/1/20 11:29 PM, Vito Jeng wrote:
> > Hi, folks,
> >
> > KIP-562(KAFKA-9445) already merged three days ago.
> >
> > I have updated KIP-216 to reflect the KIP-562. The main change is
> > to introduce a new exception `InvalidStateStorePartitionException`,
> > will be thrown when user requested partition not available.
> >
> > Please take a look and any feedback is welcome. Thanks Matthias for
> > the reminder.
> >
> > --- Vito
> >
> >
> > On Thu, Jan 23, 2020 at 2:13 PM Vito Jeng <vi...@is-land.com.tw>
> > wrote:
> >
> >> Got it, thanks Matthias.
> >>
> >> --- Vito
> >>
> >>
> >> On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax
> >> <ma...@confluent.io> wrote:
> >>
> >>> Thanks Vito.
> >>>
> >>> I am also ok with either name. Just a personal slight
> >>> preference, but not a important.
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 1/21/20 6:52 PM, Vito Jeng wrote:
> >>>> Thanks Matthias.
> >>>>
> >>>> The KIP is about InvalidStateStoreException. I pick
> >>>> `StateStoreNotAvailableException` because it may be more
> >>> intuitive
> >>>> than `StreamsNotRunningException`.
> >>>>
> >>>> No matter which one picked, it's good to me.
> >>>>
> >>>> --- Vito
> >>>>
> >>>>
> >>>> On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax
> >>>> <ma...@confluent.io> wrote:
> >>>>
> >>>>> Thanks for updating the KIP!
> >>>>>
> >>>>> One last comment/question: you kept
> >>>>> `StateStoreNotAvailableException`
> >>> in
> >>>>> favor of `StreamsNotRunningException` (to merge both as
> >>>>> suggested).
> >>>>>
> >>>>> I am wondering, if it might be better to keep
> >>>>> `StreamsNotRunningException` instead of
> >>>>> `StateStoreNotAvailableException`, because this exception
> >>>>> is thrown if Streams is in state PENDING_SHUTDOWN /
> >>>>> NOT_RUNNING / ERROR ?
> >>>>>
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>> On 1/17/20 9:56 PM, John Roesler wrote:
> >>>>>> Thanks, Vito. I've just cast my vote. -John
> >>>>>>
> >>>>>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
> >>>>>>> Hi, folks,
> >>>>>>>
> >>>>>>> Just update the KIP, please take a look.
> >>>>>>>
> >>>>>>> Thanks!
> >>>>>>>
> >>>>>>> --- Vito
> >>>>>>>
> >>>>>>>
> >>>>>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng
> >>>>>>> <vi...@is-land.com.tw>
> >>> wrote:
> >>>>>>>
> >>>>>>>> Thanks Bill, John and Matthias. Glad you guys joined
> >>>>>>>> this
> >>> discussion.
> >>>>>>>> I got a lot out of the discussion.
> >>>>>>>>
> >>>>>>>> I would like to update KIP-216 base on John's
> >>>>>>>> suggestion to remove
> >>> the
> >>>>>>>> category.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> --- Vito
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
> >>> matthias@confluent.io
> >>>>>>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>>> Nevertheless, if we omit the categorization, it’s
> >>>>>>>>>> moot.
> >>>>>>>>>
> >>>>>>>>> Ack.
> >>>>>>>>>
> >>>>>>>>> I am fine to remove the middle tier. As John
> >>>>>>>>> pointed out, it might
> >>> be
> >>>>>>>>> weird to have only one concrete exception type per
> >>>>>>>>> category. We can
> >>>>> also
> >>>>>>>>> explain in detail how to handle each exception in
> >>>>>>>>> their JavaDocs.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -Matthias
> >>>>>>>>>
> >>>>>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
> >>>>>>>>>> Vito,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the updates, the KIP LGTM.
> >>>>>>>>>>
> >>>>>>>>>> -Bill
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <
> >>> vvcephei@apache.org>
> >>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Vito,
> >>>>>>>>>>>
> >>>>>>>>>>> Haha, your archive game is on point!
> >>>>>>>>>>>
> >>>>>>>>>>> What Matthias said in that email is essentially
> >>>>>>>>>>> what I figured
> >>> was
> >>>>> the
> >>>>>>>>>>> rationale. It makes sense, but the point I was
> >>>>>>>>>>> making is that
> >>> this
> >>>>>>>>> really
> >>>>>>>>>>> doesn’t seem like a good way to structure a
> >>>>>>>>>>> production app. On
> >>> the
> >>>>>>>>> other
> >>>>>>>>>>> hand, considering the exception fatal has a
> >>>>>>>>>>> good chance of
> >>> avoiding
> >>>>> a
> >>>>>>>>>>> frustrating debug session if you just forgot to
> >>>>>>>>>>> call start.
> >>>>>>>>>>>
> >>>>>>>>>>> Nevertheless, if we omit the categorization,
> >>>>>>>>>>> it’s moot.
> >>>>>>>>>>>
> >>>>>>>>>>> It would be easy to add a categorization layer
> >>>>>>>>>>> later if we want
> >>> it,
> >>>>> but
> >>>>>>>>>>> not very easy to change it if we get it wrong.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for your consideration! -John
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>> Hi John,
> >>>>>>>>>>>>
> >>>>>>>>>>>> About `StreamsNotStartedException is strange`
> >>>>>>>>>>>> -- The original idea came from Matthias, two
> >>>>>>>>>>>> years ago. :) You can reference here:
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>
> >>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6
> c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> <https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>
> About omitting the categorization --
> >>>>>>>>>>>> It looks reasonable. I'm fine with omitting
> >>>>>>>>>>>> the categorization
> >>> but
> >>>>> not
> >>>>>>>>>>> very
> >>>>>>>>>>>> sure it is a good choice. Does any other
> >>>>>>>>>>>> folks provide opinion?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi, folks,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Just update the KIP-216, please take a look.
> >>>>>>>>>>>>
> >>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng
> >>>>>>>>>>>> <vi...@is-land.com.tw>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Hi, folks,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thank you suggestion, really appreciate it.
> >>>>>>>>>>>>> :) I understand your concern. I'll merge
> >>> StreamsNotRunningException
> >>>>> and
> >>>>>>>>>>>>> StateStoreNotAvailableException.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John
> >>>>>>>>>>>>> Roesler <
> >>> vvcephei@apache.org
> >>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hey Vito,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Yes, thanks for the KIP. Sorry the
> >>>>>>>>>>>>>> discussion has been so
> >>> long.
> >>>>>>>>>>>>>> Hopefully, we can close it out soon.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I agree we can drop
> >>>>>>>>>>>>>> StreamsNotRunningException in favor of
> >>>>>>>>>>>>>> just StateStoreNotAvailableException.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Unfortunately, I have some higher-level
> >>>>>>>>>>>>>> concerns. The value of these exceptions
> >>>>>>>>>>>>>> is that they tell you how to handle the
> >>>>>>>>>>>>>> various situations that can arise while
> >>>>>>>>>>>>>> querying a distributed data store.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Ideally, as a caller, I should be able to
> >>>>>>>>>>>>>> just catch
> >>> "retriable"
> >>>>> or
> >>>>>>>>>>>>>> "fatal" and handle them appropriately.
> >>>>>>>>>>>>>> Otherwise, there's no point in having
> >>>>>>>>>>>>>> categories, and we should just have all
> >>>>>>>>>>>>>> the exceptions extend
> >>>>>>>>>>>>>> InvalidStateStoreException.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Presently, it's not possible to tell from
> >>>>>>>>>>>>>> just the "retriable"/"fatal" distinction
> >>>>>>>>>>>>>> what to do. You  can tell from the
> >>>>>>>>>>>>>> descriptions of the various exceptions.
> >>>>>>>>>>>>>> E.g.:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Retriable: * StreamsRebalancingException:
> >>>>>>>>>>>>>> the exact same call should just be
> >>>>>>>>>>>>>> retried until the rebalance is complete *
> >>>>>>>>>>>>>> StateStoreMigratedException: the store
> >>>>>>>>>>>>>> handle is now invalid, so you need to
> >>>>>>>>>>>>>> re-discover the instance and get a new
> >>>>>>>>>>>>>> handle on that instance. In other words,
> >>>>>>>>>>>>>> the query itself may be valid, but the
> >>>>>>>>>>>>>> particular method invocation on this
> >>>>>>>>>>>>>> particular instance has encountered a
> >>>>>>>>>>>>>> fatal exception.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Fatal: * UnknownStateStoreException: this
> >>>>>>>>>>>>>> is truly fatal. No amount of retrying or
> >>>>>>>>>>>>>> re-discovering is going to get you a
> >>>>>>>>>>>>>> handle
> >>>>> on a
> >>>>>>>>>>>>>> store that doesn't exist in the cluster.
> >>>>>>>>>>>>>> * StateStoreNotAvailableException: this
> >>>>>>>>>>>>>> is actually
> >>> recoverable,
> >>>>>>>>>>>>>> since the store might exist in the
> >>>>>>>>>>>>>> cluster, but isn't
> >>>>> available
> >>>>>>>>> on
> >>>>>>>>>>>>>> this particular instance (which is shut
> >>>>>>>>>>>>>> down or whatever).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Personally, I'm not a fan of code
> >>>>>>>>>>>>>> bureaucracy, so I'm 100%
> >>> fine
> >>>>>>>>>>>>>> with omitting the categorization and just
> >>>>>>>>>>>>>> having 5 subclasses of
> >>>>>>>>>>>>>> InvalidStateStoreException. Each of them
> >>>>>>>>>>>>>> would tell you how to handle them, and
> >>>>>>>>>>>>>> it's not too many to really understand
> >>>>>>>>>>>>>> and handle each one.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If you really want to have a middle tier,
> >>>>>>>>>>>>>> I'd recommend: *
> >>>>>>>>>>>>>> RetryableStateStoreException: the exact
> >>>>>>>>>>>>>> same call should be repeated. *
> >>>>>>>>>>>>>> RecoverableStateStoreException: the store
> >>>>>>>>>>>>>> handle should be discarded and the caller
> >>>>>>>>>>>>>> should re-discover the location of the
> >>>>>>>>>>>>>> store and repeat the query on the correct
> >>>>>>>>>>>>>> instance. * FatalStateStoreException: the
> >>>>>>>>>>>>>> query/request is totally invalid and will
> >>>>>>>>>>>>>> never succeed.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> However, attempting to categorize the
> >>>>>>>>>>>>>> proposed exceptions reveals even problems
> >>>>>>>>>>>>>> with this categorization: Retriable: *
> >>>>>>>>>>>>>> StreamsRebalancingException Recoverable:
> >>>>>>>>>>>>>> * StateStoreMigratedException *
> >>>>>>>>>>>>>> StreamsNotRunningException Fatal: *
> >>>>>>>>>>>>>> UnknownStateStoreException
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> But StreamsNotStartedException is
> >>>>>>>>>>>>>> strange... It means that one code path
> >>>>>>>>>>>>>> got a handle on a specific KafkaStreams
> >>>>>>>>>>>>>> object instance and sent it a query
> >>>>>>>>>>>>>> before another code path invoked the
> >>>>>>>>>>>>>> start() method on the exact same object
> >>>>>>>>>>>>>> instance. It seems like the most likely
> >>>>>>>>>>>>>> scenario is that whoever wrote the
> >>>>>>>>>>>>>> program just forgot to call start()
> >>>>>>>>>>>>>> before querying, in which case, retrying
> >>>>>>>>>>>>>> isn't going to help, and a fatal
> >>> exception
> >>>>>>>>>>>>>> is more appropriate. I.e., it sounds like
> >>>>>>>>>>>>>> a "first 15 minutes experience" problem,
> >>>>>>>>>>>>>> and making it fatal would be more
> >>>>>>>>>>>>>> helpful. Even in a production context,
> >>>>>>>>>>>>>> there's no reason not to sequence your
> >>>>>>>>>>>>>> application startup such that you don't
> >>>>>>>>>>>>>> accept queries until after Streams is
> >>>>>>>>>>>>>> started. Thus, I guess I'd categorize it
> >>>>>>>>>>>>>> under "fatal".
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regardless of whether you make it fatal
> >>>>>>>>>>>>>> or retriable, you'd still have a whole
> >>>>>>>>>>>>>> category with only one exception in it,
> >>>>>>>>>>>>>> and the other two categories only have
> >>>>>>>>>>>>>> two exceptions. Plus, as you pointed out
> >>>>>>>>>>>>>> in the KIP, you can't get all exceptions
> >>>>>>>>>>>>>> in all cases anyway: * store() can only
> >>>>>>>>>>>>>> throw NotStarted, NotRunning, and
> >>>>>>>>>>>>>> Unknown * actual store queries can only
> >>>>>>>>>>>>>> throw Rebalancing, Migrated, and
> >>>>>>>>>>>>>> NotRunning
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thus, in practice also, there are exactly
> >>>>>>>>>>>>>> three categories and also exactly three
> >>>>>>>>>>>>>> exception types. It doesn't seem like
> >>>>>>>>>>>>>> there's a great advantage to the
> >>>>>>>>>>>>>> categories here. To avoid the
> >>>>>>>>>>>>>> categorization problem and also to
> >>>>>>>>>>>>>> clarify what exceptions can actually be
> >>>>>>>>>>>>>> thrown in different circumstances, it
> >>>>>>>>>>>>>> seems like we should just: * get rid of
> >>>>>>>>>>>>>> the middle tier and make all the
> >>>>>>>>>>>>>> exceptions extend
> >>>>>>>>>>>>>> InvalidStateStoreException * drop
> >>>>>>>>>>>>>> StateStoreNotAvailableException in favor
> >>>>>>>>>>>>>> of StreamsNotRunningException * clearly
> >>>>>>>>>>>>>> document on all public methods which
> >>>>>>>>>>>>>> exceptions need to be handled
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> How do you feel about this? Thanks,
> >>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill
> >>>>>>>>>>>>>> Bejeck wrote:
> >>>>>>>>>>>>>>> Thanks for KIP Vito.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Overall the KIP LGTM, but I'd have to
> >>>>>>>>>>>>>>> agree with others on
> >>>>> merging
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>> classes.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Since in both cases, the thread state
> >>>>>>>>>>>>>>> is in
> >>> `PENDING_SHUTDOWN ||
> >>>>>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure
> >>>>>>>>>>>>>>> how we could
> >>> distinguish
> >>>>>>>>>>> when to
> >>>>>>>>>>>>>>> use the different exceptions.  Maybe a
> >>>>>>>>>>>>>>> good middle ground would be to have a
> >>>>> detailed
> >>>>>>>>>>>>>>> exception message.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The KIP freeze is close, so I think if
> >>>>>>>>>>>>>>> we can agree on this,
> >>> we
> >>>>> can
> >>>>>>>>>>>>>> wrap up
> >>>>>>>>>>>>>>> the voting soon.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks, Bill
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM
> >>>>>>>>>>>>>>> Matthias J. Sax <
> >>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Vito,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> It's still unclear to me what the
> >>>>>>>>>>>>>>>> advantage is, to have both
> >>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> For both cased, the state is
> >>>>>>>>>>>>>>>> `PENDING_SHUTDOWN /
> >>> NOT_RUNNING /
> >>>>>>>>>>> ERROR`
> >>>>>>>>>>>>>>>> and thus, for a user point of view,
> >>>>>>>>>>>>>>>> why does it matter if
> >>> the
> >>>>>>>>>>> store is
> >>>>>>>>>>>>>>>> closed on not? I don't understand
> >>>>>>>>>>>>>>>> why/how this information
> >>>>> would
> >>>>>>>>>>> be
> >>>>>>>>>>>>>>>> useful? Do you have a concrete
> >>>>>>>>>>>>>>>> example in mind how a user
> >>> would
> >>>>>>>>>>> react
> >>>>>>>>>>>>>>>> differently to both exceptions?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> @Vinoth: about
> >>>>>>>>>>>>>>>> `StreamsRebalancingException` -- to
> >>>>>>>>>>>>>>>> me, it
> >>> seems
> >>>>>>>>>>> best
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> actually do this on a per-query
> >>>>>>>>>>>>>>>> basis, ie, have an overload
> >>>>>>>>>>>>>>>> `KafkaStreams#store(...)` that takes
> >>>>>>>>>>>>>>>> a boolean flag that
> >>> allow
> >>>>> to
> >>>>>>>>>>>>>>>> _disable_ the exception and opt-in to
> >>>>>>>>>>>>>>>> query a active store
> >>>>> during
> >>>>>>>>>>>>>>>> recovery. However, as KIP-535
> >>>>>>>>>>>>>>>> actually introduces this
> >>> change
> >>>>> in
> >>>>>>>>>>>>>>>> behavior, I think KIP-216 should not
> >>>>>>>>>>>>>>>> cover this, but KIP-535
> >>>>>>>>>>> should be
> >>>>>>>>>>>>>>>> updated. I'll follow up on the other
> >>>>>>>>>>>>>>>> KIP thread to raise
> >>> this
> >>>>>>>>>>> point.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>> Hi, Matthias & Vinoth,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> What is still unclear to me is,
> >>>>>>>>>>>>>>>>>> what we gain by having
> >>> both
> >>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>> `StateStoreNotAvailableException`. Both
> >>>>>>>>>>>>>>>>>> exception are thrown when
> >>>>>>>>>>>>>>>>>> KafkaStreams is in state
> >>>>>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a
> >>>>>>>>>>>>>>>>>> user what do I gain to
> >>> know
> >>>>>>>>>>> if the
> >>>>>>>>>>>>>>>>>> state store is closed on not -- I
> >>>>>>>>>>>>>>>>>> can't query it anyway?
> >>>>> Maybe
> >>>>>>>>>>> I
> >>>>>>>>>>>>>> miss
> >>>>>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Yes, both
> >>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>>>>> are fatal exception. But
> >>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>>>>> is fatal exception
> >>> about
> >>>>>>>>>>> state
> >>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>> related. I think it would be
> >>>>>>>>>>>>>>>>> helpful that if user need to
> >>> distinguish
> >>>>>>>>>>> these
> >>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>> different case to handle it.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I'm not very sure, does that make
> >>>>>>>>>>>>>>>>> sense?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM
> >>>>>>>>>>>>>>>>> Vinoth Chandar <
> >>>>>>>>>>> vinoth@apache.org>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> +1 on merging
> >>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`,
> >>>>>>>>>>>>>>>>>> both exceptions are
> >>> fatal
> >>>>>>>>>>>>>> anyway. IMO
> >>>>>>>>>>>>>>>>>> its best to have these exceptions
> >>>>>>>>>>>>>>>>>> be about the state store
> >>>>>>>>>>> (and not
> >>>>>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>>> state), to easier understanding.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Additionally, KIP-535 allows for
> >>>>>>>>>>>>>>>>>> querying of state stores
> >>> in
> >>>>>>>>>>>>>> rebalancing
> >>>>>>>>>>>>>>>>>> state. So do we need the
> >>>>>>>>>>>>>>>>>> StreamsRebalancingException?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias
> >>>>>>>>>>>>>>>>>> J. Sax" <
> >>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>> Sorry that I dropped the ball
> >>>>>>>>>>>>>>>>>>> on this...
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for updating the KIP.
> >>>>>>>>>>>>>>>>>>> Overall LGTM now. Feel free
> >>> to
> >>>>>>>>>>> start
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> VOTE
> >>>>>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> What is still unclear to me is,
> >>>>>>>>>>>>>>>>>>> what we gain by having
> >>> both
> >>>>>>>>>>>>>>>>>>> `StreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>> `StateStoreNotAvailableException`.
> >>>>>>>>>>>>>>>> Both
> >>>>>>>>>>>>>>>>>>> exception are thrown when
> >>>>>>>>>>>>>>>>>>> KafkaStreams is in state
> >>>>>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as
> >>>>>>>>>>>>>>>>>>> a user what do I gain to
> >>> know
> >>>>>>>>>>> if
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> state store is closed on not --
> >>>>>>>>>>>>>>>>>>> I can't query it anyway?
> >>>>>>>>>>> Maybe I
> >>>>>>>>>>>>>> miss
> >>>>>>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>> Sorry for the late reply,
> >>>>>>>>>>>>>>>>>>>> thanks for the review.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> Why is it only thrown if the state is REBALANCING? A
> >>> store
> >>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>> migrated during a
> >>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
> >>>>>>>>>>>>>>>>>>>>> Streams might
> >>>>> resume
> >>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>> RUNNING state and afterward
> >>>>>>>>>>>>>>>>>>>>> somebody tries to use an
> >>> old
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>>>> Also, if state is
> >>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
> >>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
> >>>>>>>>>>>>>>>>>>>>> Hence, I think
> >>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>>>>> does only make sense
> >>> during
> >>>>>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thank you point this, already
> >>>>>>>>>>>>>>>>>>>> updated.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Why do we need to distinguish
> >>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> `KafkaStreamsNotRunningException` may be caused by
> >>> various
> >>>>>>>>>>>>>> reasons, I
> >>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>> it would be helpful that the
> >>>>>>>>>>>>>>>>>>>> user can distinguish whether
> >>>>>>>>>>>>>>>>>>>> it is caused by the state
> >>>>> store
> >>>>>>>>>>>>>> closed.
> >>>>>>>>>>>>>>>>>>>> (Maybe I am wrong...)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Last, why do we distinguish
> >>>>>>>>>>>>>>>>>>>> between `KafkaStreams`
> >>> instance
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it
> >>>>>>>>>>>>>>>>>>>>> seems we should always
> >>> refer to
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>>>>> because that is the level
> >>>>>>>>>>>>>>>>>>>>> of granularity in which we
> >>>>>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>>>>> IQ atm.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Totally agree. Do you mean
> >>>>>>>>>>>>>>>>>>>> the naming of state store
> >>>>>>>>>>> exceptions?
> >>>>>>>>>>>>>>>>>>>> I don't have special reason
> >>>>>>>>>>>>>>>>>>>> to distinguish these two.
> >>>>>>>>>>>>>>>>>>>> Your suggestion look more
> >>>>>>>>>>>>>>>>>>>> reasonable for the exception
> >>>>>>>>>>> naming.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Last, for
> >>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
> >>>>>>>>>>>>>>>>>>>> I would add
> >>> that a
> >>>>>>>>>>> user
> >>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> rediscover the store and
> >>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
> >>>>>>>>>>>>>>>>>>>>> the
> >>> store
> >>>>>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>>>>> invalid and a new store
> >>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
> >>>>>>>>>>>>>>>>>>>>> That
> >>> is
> >>>>> a
> >>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
> >>>>>>>>>>>>>>>>>>>>> that allows for
> >>>>>>>>>>> "blind"
> >>>>>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>>>>> that either resolve (if the
> >>>>>>>>>>>>>>>>>>>>> store is still on the same
> >>>>>>>>>>> instance
> >>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
> >>>>>>>>>>>>>>>>>>>>> changes to
> >>>>>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> store was migrated away
> >>>>>>>>>>>>>>>>>>>>> during rebalancing).
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The KIP already updated,
> >>>>>>>>>>>>>>>>>>>> please take a look. :)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48
> >>>>>>>>>>>>>>>>>>>> PM Matthias J. Sax <
> >>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Any update on this KIP?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM,
> >>>>>>>>>>>>>>>>>>>>> Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply.
> >>>>>>>>>>>>>>>>>>>>>> The 2.4 deadline kept us
> >>> quite
> >>>>>>>>>>> busy.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> Why is it only thrown if the state is REBALANCING? A
> >>>>> store
> >>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>>> migrated during a
> >>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
> >>>>>>>>>>>>>>>>>>>>>> Streams might
> >>>>> resume
> >>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>> RUNNING state and
> >>>>>>>>>>>>>>>>>>>>>> afterward somebody tries
> >>>>>>>>>>>>>>>>>>>>>> to use an
> >>> old
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>>>>> Also, if state is
> >>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
> >>>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
> >>>>>>>>>>>>>>>>>>>>>> Hence, I think
> >>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>>>>>> does only make sense
> >>> during
> >>>>>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Why do we need to
> >>>>>>>>>>>>>>>>>>>>>> distinguish between
> >>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Last, why do we
> >>>>>>>>>>>>>>>>>>>>>> distinguish between
> >>>>>>>>>>>>>>>>>>>>>> `KafkaStreams`
> >>>>>>>>>>> instance and
> >>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me,
> >>>>>>>>>>>>>>>>>>>>>> it seems we should
> >>>>>>>>>>>>>>>>>>>>>> always
> >>> refer
> >>>>> to
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>>>>>> because that is the level
> >>>>>>>>>>>>>>>>>>>>>> of granularity in which
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>>>>> IQ
> >>>>>>>>>>>>>>>>>>>>> atm.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Last, for
> >>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
> >>>>>>>>>>>>>>>>>>>>>> I would add
> >>>>> that a
> >>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>> rediscover the store and
> >>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>> store
> >>>>>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>>>>>> invalid and a new store
> >>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
> >>>>>>>>>>>>>>>>>>>>>> That
> >>>>> is a
> >>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
> >>>>>>>>>>>>>>>>>>>>>> that allows for
> >>>>>>>>>>> "blind"
> >>>>>>>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>>>>>> that either resolve (if
> >>>>>>>>>>>>>>>>>>>>>> the store is still on the
> >>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>> instance
> >>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
> >>>>>>>>>>>>>>>>>>>>>> changes to
> >>>>>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> store was migrated away
> >>>>>>>>>>>>>>>>>>>>>> during rebalancing).
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito
> >>>>>>>>>>>>>>>>>>>>>> Jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>> My bad. The short link
> >>>>>>>>>>>>>>>>>>>>>>> `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >>>
> >>>>>>>>>>>>>>>>>>>>>>>
> <https://shorturl.at/CDNT9>
> >>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Please use the
> >>>>>>>>>>>>>>>>>>>>>>> following instead:
> >>>>>>>>>>> https://shorturl.at/bkKQU
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at
> >>>>>>>>>>>>>>>>>>>>>>> 10:53 AM Vito Jeng <
> >>>>>>>>>>>>>> vito@is-land.com.tw>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> Thank you for explanation. I agree with your
> >>> opinion.
> >>>>>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()`
> >>>>>>>>>>>>>>>>>>>>>>>> would never throw
> >>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> For the case that corresponding thread crashes
> >>> after we
> >>>>>>>>>>>>>> handed out
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> store handle. We may
> >>>>>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>> when stream
> >>> thread is
> >>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> running(
> >>>>>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9)
> >>>>>>>>>>>>>>>>>>>>>>>> or throw
> >>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>> store is
> >>>>>>>>>>>>>>>>>>>>>>>> closed(https://shorturl.at/hrvAN).
> >>>>>>>>>>>>>>>>>>>>>>>> So I
> >>> think
> >>>>>>>>>>> we
> >>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> add a new type for
> >>>>>>>>>>>>>>>>>>>>>>>> this case. Does that
> >>>>>>>>>>>>>>>>>>>>>>>> make sense?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>>> vs
> >>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> I understand your point. I rename
> >>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> About check unknown state store names:
> >>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> >>>>>>>>>>>>>>>>>>>>>>>> hint. I add a new
> >>>>>>>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>> `UnknownStateStoreException`
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> this case.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Also, we should
> >>>>>>>>>>>>>>>>>>>>>>>>> still have fatal
> >>>>>>>>>>>>>>>>>>>>>>>>> exception
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>>>> Not sure why you
> >>>>>>>>>>> remove
> >>>>>>>>>>>>>> it?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thank you point this,
> >>>>>>>>>>>>>>>>>>>>>>>> already add it
> >>>>>>>>>>>>>>>>>>>>>>>> again.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> The KIP already
> >>>>>>>>>>>>>>>>>>>>>>>> updated, please take
> >>>>>>>>>>>>>>>>>>>>>>>> a look.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>
> >>>
> >
> -----BEGIN PGP SIGNATURE-----
>
> iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl45vCwACgkQO4miYXKq
> /OhT4Q//elvNj2BRIMd7SqyQhpa6sSq4k2iG3wFlr/B2xkmkZPWUICt3SpCisIxG
> RlP5ml7Mi8IcWv9jmMux9C1NpftboLudxIUcun/I4cP0T3S7ytXfzdO+V1WuD9xe
> hfeqMwJTytusEv5VO3MyY+jYGqmPJrPJRViZ2Uwsj3Ojk0SZRb/m+b6ah6BGMh0E
> nlUIux7cabg1OZ/ee0x21hcAB32lVIRfPLKZeHCTHuYmlW76X4pXhFxOm69Pumtj
> ciVMr/pl7B+XiMk0C2Po04zQcP8+5/O7LYU4e8ha12NtuNuDTlQLGUb+S9Qrbxb2
> xdFrlC81RROmIdIOTLiJeeBqhzd19llDgMhBf4spJMOeLn7359PX6r/9tDqonJrQ
> wpSp4S54MDYfTPHUPqA6u/FsE0BX4EMU71ckC9rXVRRUnQ8A8cMk/6qWlRu8NvxQ
> IiYuSo48UENlcTHxXC/rzIDp54gvtv2iml8QcOg6tS+hzhAI4yoY+1w4pnjPmRo0
> 4D1hnq5bd1SLrGSGcCxVbN0jtwgkcr50HBb1UkIDRndqapfKwZOMV65tIwTvxaaZ
> r2QPYyPd6ZDgeMdy0r94z0SMwatqGCmJD9EQmSmLBulemrvYxPGUevQls+WgE4WE
> zc3LtoyfdlXlkttTU0Q/dm2H9OU4s3QO+6PEdC+Qwi3aCKKhdUQ=
> =4EMC
> -----END PGP SIGNATURE-----
>
>
>
>
>  

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Hi, Matthias and Navinder,

I have a question about the valid partition in
StreamThreadStateStoreProvider.

In the StreamThreadStateStoreProvider#createKeyTaskId(storeName, partition):
https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L103

We pass an integer as partition and then use this partition to create
TaskId instance in the topic group while loop. How do we make sure the
partition is valid? If we pass a correct storeName and a invalid partition
into createKeyTaskId() , it still looks can be created a new TaskId and
would not throw InvalidStateStorePartitionException.

I guess this would cause a NullPointerException at line #62 because this
keyTaskId cannot found in the task list.
https://github.com/apache/kafka/blob/bbfecaef725456f648f03530d26a5395042966fa/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java#L62

Does this right? or there something wrong with me?

---
Vito


On Wed, Feb 5, 2020 at 2:53 AM Navinder Brar
<na...@yahoo.com.invalid> wrote:

> Thanks Vito, for incorporating this. Makes sense.
>
> -Navinder
>
>
> On Wednesday, February 5, 2020, 12:17 AM, Matthias J. Sax <
> mjsax@apache.org> wrote:
>
> -----BEGIN PGP SIGNED MESSAGE-----
> Hash: SHA512
>
> Thanks Vito!
>
> That makes sense to me.
>
>
> On 2/1/20 11:29 PM, Vito Jeng wrote:
> > Hi, folks,
> >
> > KIP-562(KAFKA-9445) already merged three days ago.
> >
> > I have updated KIP-216 to reflect the KIP-562. The main change is
> > to introduce a new exception `InvalidStateStorePartitionException`,
> > will be thrown when user requested partition not available.
> >
> > Please take a look and any feedback is welcome. Thanks Matthias for
> > the reminder.
> >
> > --- Vito
> >
> >
> > On Thu, Jan 23, 2020 at 2:13 PM Vito Jeng <vi...@is-land.com.tw>
> > wrote:
> >
> >> Got it, thanks Matthias.
> >>
> >> --- Vito
> >>
> >>
> >> On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax
> >> <ma...@confluent.io> wrote:
> >>
> >>> Thanks Vito.
> >>>
> >>> I am also ok with either name. Just a personal slight
> >>> preference, but not a important.
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 1/21/20 6:52 PM, Vito Jeng wrote:
> >>>> Thanks Matthias.
> >>>>
> >>>> The KIP is about InvalidStateStoreException. I pick
> >>>> `StateStoreNotAvailableException` because it may be more
> >>> intuitive
> >>>> than `StreamsNotRunningException`.
> >>>>
> >>>> No matter which one picked, it's good to me.
> >>>>
> >>>> --- Vito
> >>>>
> >>>>
> >>>> On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax
> >>>> <ma...@confluent.io> wrote:
> >>>>
> >>>>> Thanks for updating the KIP!
> >>>>>
> >>>>> One last comment/question: you kept
> >>>>> `StateStoreNotAvailableException`
> >>> in
> >>>>> favor of `StreamsNotRunningException` (to merge both as
> >>>>> suggested).
> >>>>>
> >>>>> I am wondering, if it might be better to keep
> >>>>> `StreamsNotRunningException` instead of
> >>>>> `StateStoreNotAvailableException`, because this exception
> >>>>> is thrown if Streams is in state PENDING_SHUTDOWN /
> >>>>> NOT_RUNNING / ERROR ?
> >>>>>
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>> On 1/17/20 9:56 PM, John Roesler wrote:
> >>>>>> Thanks, Vito. I've just cast my vote. -John
> >>>>>>
> >>>>>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
> >>>>>>> Hi, folks,
> >>>>>>>
> >>>>>>> Just update the KIP, please take a look.
> >>>>>>>
> >>>>>>> Thanks!
> >>>>>>>
> >>>>>>> --- Vito
> >>>>>>>
> >>>>>>>
> >>>>>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng
> >>>>>>> <vi...@is-land.com.tw>
> >>> wrote:
> >>>>>>>
> >>>>>>>> Thanks Bill, John and Matthias. Glad you guys joined
> >>>>>>>> this
> >>> discussion.
> >>>>>>>> I got a lot out of the discussion.
> >>>>>>>>
> >>>>>>>> I would like to update KIP-216 base on John's
> >>>>>>>> suggestion to remove
> >>> the
> >>>>>>>> category.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> --- Vito
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
> >>> matthias@confluent.io
> >>>>>>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>>> Nevertheless, if we omit the categorization, it’s
> >>>>>>>>>> moot.
> >>>>>>>>>
> >>>>>>>>> Ack.
> >>>>>>>>>
> >>>>>>>>> I am fine to remove the middle tier. As John
> >>>>>>>>> pointed out, it might
> >>> be
> >>>>>>>>> weird to have only one concrete exception type per
> >>>>>>>>> category. We can
> >>>>> also
> >>>>>>>>> explain in detail how to handle each exception in
> >>>>>>>>> their JavaDocs.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -Matthias
> >>>>>>>>>
> >>>>>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
> >>>>>>>>>> Vito,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the updates, the KIP LGTM.
> >>>>>>>>>>
> >>>>>>>>>> -Bill
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <
> >>> vvcephei@apache.org>
> >>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Vito,
> >>>>>>>>>>>
> >>>>>>>>>>> Haha, your archive game is on point!
> >>>>>>>>>>>
> >>>>>>>>>>> What Matthias said in that email is essentially
> >>>>>>>>>>> what I figured
> >>> was
> >>>>> the
> >>>>>>>>>>> rationale. It makes sense, but the point I was
> >>>>>>>>>>> making is that
> >>> this
> >>>>>>>>> really
> >>>>>>>>>>> doesn’t seem like a good way to structure a
> >>>>>>>>>>> production app. On
> >>> the
> >>>>>>>>> other
> >>>>>>>>>>> hand, considering the exception fatal has a
> >>>>>>>>>>> good chance of
> >>> avoiding
> >>>>> a
> >>>>>>>>>>> frustrating debug session if you just forgot to
> >>>>>>>>>>> call start.
> >>>>>>>>>>>
> >>>>>>>>>>> Nevertheless, if we omit the categorization,
> >>>>>>>>>>> it’s moot.
> >>>>>>>>>>>
> >>>>>>>>>>> It would be easy to add a categorization layer
> >>>>>>>>>>> later if we want
> >>> it,
> >>>>> but
> >>>>>>>>>>> not very easy to change it if we get it wrong.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for your consideration! -John
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>> Hi John,
> >>>>>>>>>>>>
> >>>>>>>>>>>> About `StreamsNotStartedException is strange`
> >>>>>>>>>>>> -- The original idea came from Matthias, two
> >>>>>>>>>>>> years ago. :) You can reference here:
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>
> >>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6
> c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> <https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>
> About omitting the categorization --
> >>>>>>>>>>>> It looks reasonable. I'm fine with omitting
> >>>>>>>>>>>> the categorization
> >>> but
> >>>>> not
> >>>>>>>>>>> very
> >>>>>>>>>>>> sure it is a good choice. Does any other
> >>>>>>>>>>>> folks provide opinion?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi, folks,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Just update the KIP-216, please take a look.
> >>>>>>>>>>>>
> >>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng
> >>>>>>>>>>>> <vi...@is-land.com.tw>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Hi, folks,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thank you suggestion, really appreciate it.
> >>>>>>>>>>>>> :) I understand your concern. I'll merge
> >>> StreamsNotRunningException
> >>>>> and
> >>>>>>>>>>>>> StateStoreNotAvailableException.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John
> >>>>>>>>>>>>> Roesler <
> >>> vvcephei@apache.org
> >>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hey Vito,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Yes, thanks for the KIP. Sorry the
> >>>>>>>>>>>>>> discussion has been so
> >>> long.
> >>>>>>>>>>>>>> Hopefully, we can close it out soon.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I agree we can drop
> >>>>>>>>>>>>>> StreamsNotRunningException in favor of
> >>>>>>>>>>>>>> just StateStoreNotAvailableException.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Unfortunately, I have some higher-level
> >>>>>>>>>>>>>> concerns. The value of these exceptions
> >>>>>>>>>>>>>> is that they tell you how to handle the
> >>>>>>>>>>>>>> various situations that can arise while
> >>>>>>>>>>>>>> querying a distributed data store.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Ideally, as a caller, I should be able to
> >>>>>>>>>>>>>> just catch
> >>> "retriable"
> >>>>> or
> >>>>>>>>>>>>>> "fatal" and handle them appropriately.
> >>>>>>>>>>>>>> Otherwise, there's no point in having
> >>>>>>>>>>>>>> categories, and we should just have all
> >>>>>>>>>>>>>> the exceptions extend
> >>>>>>>>>>>>>> InvalidStateStoreException.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Presently, it's not possible to tell from
> >>>>>>>>>>>>>> just the "retriable"/"fatal" distinction
> >>>>>>>>>>>>>> what to do. You  can tell from the
> >>>>>>>>>>>>>> descriptions of the various exceptions.
> >>>>>>>>>>>>>> E.g.:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Retriable: * StreamsRebalancingException:
> >>>>>>>>>>>>>> the exact same call should just be
> >>>>>>>>>>>>>> retried until the rebalance is complete *
> >>>>>>>>>>>>>> StateStoreMigratedException: the store
> >>>>>>>>>>>>>> handle is now invalid, so you need to
> >>>>>>>>>>>>>> re-discover the instance and get a new
> >>>>>>>>>>>>>> handle on that instance. In other words,
> >>>>>>>>>>>>>> the query itself may be valid, but the
> >>>>>>>>>>>>>> particular method invocation on this
> >>>>>>>>>>>>>> particular instance has encountered a
> >>>>>>>>>>>>>> fatal exception.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Fatal: * UnknownStateStoreException: this
> >>>>>>>>>>>>>> is truly fatal. No amount of retrying or
> >>>>>>>>>>>>>> re-discovering is going to get you a
> >>>>>>>>>>>>>> handle
> >>>>> on a
> >>>>>>>>>>>>>> store that doesn't exist in the cluster.
> >>>>>>>>>>>>>> * StateStoreNotAvailableException: this
> >>>>>>>>>>>>>> is actually
> >>> recoverable,
> >>>>>>>>>>>>>> since the store might exist in the
> >>>>>>>>>>>>>> cluster, but isn't
> >>>>> available
> >>>>>>>>> on
> >>>>>>>>>>>>>> this particular instance (which is shut
> >>>>>>>>>>>>>> down or whatever).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Personally, I'm not a fan of code
> >>>>>>>>>>>>>> bureaucracy, so I'm 100%
> >>> fine
> >>>>>>>>>>>>>> with omitting the categorization and just
> >>>>>>>>>>>>>> having 5 subclasses of
> >>>>>>>>>>>>>> InvalidStateStoreException. Each of them
> >>>>>>>>>>>>>> would tell you how to handle them, and
> >>>>>>>>>>>>>> it's not too many to really understand
> >>>>>>>>>>>>>> and handle each one.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If you really want to have a middle tier,
> >>>>>>>>>>>>>> I'd recommend: *
> >>>>>>>>>>>>>> RetryableStateStoreException: the exact
> >>>>>>>>>>>>>> same call should be repeated. *
> >>>>>>>>>>>>>> RecoverableStateStoreException: the store
> >>>>>>>>>>>>>> handle should be discarded and the caller
> >>>>>>>>>>>>>> should re-discover the location of the
> >>>>>>>>>>>>>> store and repeat the query on the correct
> >>>>>>>>>>>>>> instance. * FatalStateStoreException: the
> >>>>>>>>>>>>>> query/request is totally invalid and will
> >>>>>>>>>>>>>> never succeed.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> However, attempting to categorize the
> >>>>>>>>>>>>>> proposed exceptions reveals even problems
> >>>>>>>>>>>>>> with this categorization: Retriable: *
> >>>>>>>>>>>>>> StreamsRebalancingException Recoverable:
> >>>>>>>>>>>>>> * StateStoreMigratedException *
> >>>>>>>>>>>>>> StreamsNotRunningException Fatal: *
> >>>>>>>>>>>>>> UnknownStateStoreException
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> But StreamsNotStartedException is
> >>>>>>>>>>>>>> strange... It means that one code path
> >>>>>>>>>>>>>> got a handle on a specific KafkaStreams
> >>>>>>>>>>>>>> object instance and sent it a query
> >>>>>>>>>>>>>> before another code path invoked the
> >>>>>>>>>>>>>> start() method on the exact same object
> >>>>>>>>>>>>>> instance. It seems like the most likely
> >>>>>>>>>>>>>> scenario is that whoever wrote the
> >>>>>>>>>>>>>> program just forgot to call start()
> >>>>>>>>>>>>>> before querying, in which case, retrying
> >>>>>>>>>>>>>> isn't going to help, and a fatal
> >>> exception
> >>>>>>>>>>>>>> is more appropriate. I.e., it sounds like
> >>>>>>>>>>>>>> a "first 15 minutes experience" problem,
> >>>>>>>>>>>>>> and making it fatal would be more
> >>>>>>>>>>>>>> helpful. Even in a production context,
> >>>>>>>>>>>>>> there's no reason not to sequence your
> >>>>>>>>>>>>>> application startup such that you don't
> >>>>>>>>>>>>>> accept queries until after Streams is
> >>>>>>>>>>>>>> started. Thus, I guess I'd categorize it
> >>>>>>>>>>>>>> under "fatal".
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regardless of whether you make it fatal
> >>>>>>>>>>>>>> or retriable, you'd still have a whole
> >>>>>>>>>>>>>> category with only one exception in it,
> >>>>>>>>>>>>>> and the other two categories only have
> >>>>>>>>>>>>>> two exceptions. Plus, as you pointed out
> >>>>>>>>>>>>>> in the KIP, you can't get all exceptions
> >>>>>>>>>>>>>> in all cases anyway: * store() can only
> >>>>>>>>>>>>>> throw NotStarted, NotRunning, and
> >>>>>>>>>>>>>> Unknown * actual store queries can only
> >>>>>>>>>>>>>> throw Rebalancing, Migrated, and
> >>>>>>>>>>>>>> NotRunning
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thus, in practice also, there are exactly
> >>>>>>>>>>>>>> three categories and also exactly three
> >>>>>>>>>>>>>> exception types. It doesn't seem like
> >>>>>>>>>>>>>> there's a great advantage to the
> >>>>>>>>>>>>>> categories here. To avoid the
> >>>>>>>>>>>>>> categorization problem and also to
> >>>>>>>>>>>>>> clarify what exceptions can actually be
> >>>>>>>>>>>>>> thrown in different circumstances, it
> >>>>>>>>>>>>>> seems like we should just: * get rid of
> >>>>>>>>>>>>>> the middle tier and make all the
> >>>>>>>>>>>>>> exceptions extend
> >>>>>>>>>>>>>> InvalidStateStoreException * drop
> >>>>>>>>>>>>>> StateStoreNotAvailableException in favor
> >>>>>>>>>>>>>> of StreamsNotRunningException * clearly
> >>>>>>>>>>>>>> document on all public methods which
> >>>>>>>>>>>>>> exceptions need to be handled
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> How do you feel about this? Thanks,
> >>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill
> >>>>>>>>>>>>>> Bejeck wrote:
> >>>>>>>>>>>>>>> Thanks for KIP Vito.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Overall the KIP LGTM, but I'd have to
> >>>>>>>>>>>>>>> agree with others on
> >>>>> merging
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>> classes.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Since in both cases, the thread state
> >>>>>>>>>>>>>>> is in
> >>> `PENDING_SHUTDOWN ||
> >>>>>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure
> >>>>>>>>>>>>>>> how we could
> >>> distinguish
> >>>>>>>>>>> when to
> >>>>>>>>>>>>>>> use the different exceptions.  Maybe a
> >>>>>>>>>>>>>>> good middle ground would be to have a
> >>>>> detailed
> >>>>>>>>>>>>>>> exception message.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The KIP freeze is close, so I think if
> >>>>>>>>>>>>>>> we can agree on this,
> >>> we
> >>>>> can
> >>>>>>>>>>>>>> wrap up
> >>>>>>>>>>>>>>> the voting soon.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks, Bill
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM
> >>>>>>>>>>>>>>> Matthias J. Sax <
> >>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Vito,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> It's still unclear to me what the
> >>>>>>>>>>>>>>>> advantage is, to have both
> >>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> For both cased, the state is
> >>>>>>>>>>>>>>>> `PENDING_SHUTDOWN /
> >>> NOT_RUNNING /
> >>>>>>>>>>> ERROR`
> >>>>>>>>>>>>>>>> and thus, for a user point of view,
> >>>>>>>>>>>>>>>> why does it matter if
> >>> the
> >>>>>>>>>>> store is
> >>>>>>>>>>>>>>>> closed on not? I don't understand
> >>>>>>>>>>>>>>>> why/how this information
> >>>>> would
> >>>>>>>>>>> be
> >>>>>>>>>>>>>>>> useful? Do you have a concrete
> >>>>>>>>>>>>>>>> example in mind how a user
> >>> would
> >>>>>>>>>>> react
> >>>>>>>>>>>>>>>> differently to both exceptions?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> @Vinoth: about
> >>>>>>>>>>>>>>>> `StreamsRebalancingException` -- to
> >>>>>>>>>>>>>>>> me, it
> >>> seems
> >>>>>>>>>>> best
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> actually do this on a per-query
> >>>>>>>>>>>>>>>> basis, ie, have an overload
> >>>>>>>>>>>>>>>> `KafkaStreams#store(...)` that takes
> >>>>>>>>>>>>>>>> a boolean flag that
> >>> allow
> >>>>> to
> >>>>>>>>>>>>>>>> _disable_ the exception and opt-in to
> >>>>>>>>>>>>>>>> query a active store
> >>>>> during
> >>>>>>>>>>>>>>>> recovery. However, as KIP-535
> >>>>>>>>>>>>>>>> actually introduces this
> >>> change
> >>>>> in
> >>>>>>>>>>>>>>>> behavior, I think KIP-216 should not
> >>>>>>>>>>>>>>>> cover this, but KIP-535
> >>>>>>>>>>> should be
> >>>>>>>>>>>>>>>> updated. I'll follow up on the other
> >>>>>>>>>>>>>>>> KIP thread to raise
> >>> this
> >>>>>>>>>>> point.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>> Hi, Matthias & Vinoth,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> What is still unclear to me is,
> >>>>>>>>>>>>>>>>>> what we gain by having
> >>> both
> >>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>> `StateStoreNotAvailableException`. Both
> >>>>>>>>>>>>>>>>>> exception are thrown when
> >>>>>>>>>>>>>>>>>> KafkaStreams is in state
> >>>>>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a
> >>>>>>>>>>>>>>>>>> user what do I gain to
> >>> know
> >>>>>>>>>>> if the
> >>>>>>>>>>>>>>>>>> state store is closed on not -- I
> >>>>>>>>>>>>>>>>>> can't query it anyway?
> >>>>> Maybe
> >>>>>>>>>>> I
> >>>>>>>>>>>>>> miss
> >>>>>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Yes, both
> >>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>>>>> are fatal exception. But
> >>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
> >>>>>>>>>>>>>>>>> is fatal exception
> >>> about
> >>>>>>>>>>> state
> >>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>> related. I think it would be
> >>>>>>>>>>>>>>>>> helpful that if user need to
> >>> distinguish
> >>>>>>>>>>> these
> >>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>> different case to handle it.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I'm not very sure, does that make
> >>>>>>>>>>>>>>>>> sense?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM
> >>>>>>>>>>>>>>>>> Vinoth Chandar <
> >>>>>>>>>>> vinoth@apache.org>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> +1 on merging
> >>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`,
> >>>>>>>>>>>>>>>>>> both exceptions are
> >>> fatal
> >>>>>>>>>>>>>> anyway. IMO
> >>>>>>>>>>>>>>>>>> its best to have these exceptions
> >>>>>>>>>>>>>>>>>> be about the state store
> >>>>>>>>>>> (and not
> >>>>>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>>> state), to easier understanding.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Additionally, KIP-535 allows for
> >>>>>>>>>>>>>>>>>> querying of state stores
> >>> in
> >>>>>>>>>>>>>> rebalancing
> >>>>>>>>>>>>>>>>>> state. So do we need the
> >>>>>>>>>>>>>>>>>> StreamsRebalancingException?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias
> >>>>>>>>>>>>>>>>>> J. Sax" <
> >>>>>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>> Sorry that I dropped the ball
> >>>>>>>>>>>>>>>>>>> on this...
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for updating the KIP.
> >>>>>>>>>>>>>>>>>>> Overall LGTM now. Feel free
> >>> to
> >>>>>>>>>>> start
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> VOTE
> >>>>>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> What is still unclear to me is,
> >>>>>>>>>>>>>>>>>>> what we gain by having
> >>> both
> >>>>>>>>>>>>>>>>>>> `StreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>> `StateStoreNotAvailableException`.
> >>>>>>>>>>>>>>>> Both
> >>>>>>>>>>>>>>>>>>> exception are thrown when
> >>>>>>>>>>>>>>>>>>> KafkaStreams is in state
> >>>>>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as
> >>>>>>>>>>>>>>>>>>> a user what do I gain to
> >>> know
> >>>>>>>>>>> if
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> state store is closed on not --
> >>>>>>>>>>>>>>>>>>> I can't query it anyway?
> >>>>>>>>>>> Maybe I
> >>>>>>>>>>>>>> miss
> >>>>>>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>> Sorry for the late reply,
> >>>>>>>>>>>>>>>>>>>> thanks for the review.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> Why is it only thrown if the state is REBALANCING? A
> >>> store
> >>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>> migrated during a
> >>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
> >>>>>>>>>>>>>>>>>>>>> Streams might
> >>>>> resume
> >>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>> RUNNING state and afterward
> >>>>>>>>>>>>>>>>>>>>> somebody tries to use an
> >>> old
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>>>> Also, if state is
> >>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
> >>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
> >>>>>>>>>>>>>>>>>>>>> Hence, I think
> >>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>>>>> does only make sense
> >>> during
> >>>>>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thank you point this, already
> >>>>>>>>>>>>>>>>>>>> updated.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Why do we need to distinguish
> >>>>>>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> `KafkaStreamsNotRunningException` may be caused by
> >>> various
> >>>>>>>>>>>>>> reasons, I
> >>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>> it would be helpful that the
> >>>>>>>>>>>>>>>>>>>> user can distinguish whether
> >>>>>>>>>>>>>>>>>>>> it is caused by the state
> >>>>> store
> >>>>>>>>>>>>>> closed.
> >>>>>>>>>>>>>>>>>>>> (Maybe I am wrong...)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Last, why do we distinguish
> >>>>>>>>>>>>>>>>>>>> between `KafkaStreams`
> >>> instance
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it
> >>>>>>>>>>>>>>>>>>>>> seems we should always
> >>> refer to
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>>>>> because that is the level
> >>>>>>>>>>>>>>>>>>>>> of granularity in which we
> >>>>>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>>>>> IQ atm.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Totally agree. Do you mean
> >>>>>>>>>>>>>>>>>>>> the naming of state store
> >>>>>>>>>>> exceptions?
> >>>>>>>>>>>>>>>>>>>> I don't have special reason
> >>>>>>>>>>>>>>>>>>>> to distinguish these two.
> >>>>>>>>>>>>>>>>>>>> Your suggestion look more
> >>>>>>>>>>>>>>>>>>>> reasonable for the exception
> >>>>>>>>>>> naming.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Last, for
> >>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
> >>>>>>>>>>>>>>>>>>>> I would add
> >>> that a
> >>>>>>>>>>> user
> >>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> rediscover the store and
> >>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
> >>>>>>>>>>>>>>>>>>>>> the
> >>> store
> >>>>>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>>>>> invalid and a new store
> >>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
> >>>>>>>>>>>>>>>>>>>>> That
> >>> is
> >>>>> a
> >>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
> >>>>>>>>>>>>>>>>>>>>> that allows for
> >>>>>>>>>>> "blind"
> >>>>>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>>>>> that either resolve (if the
> >>>>>>>>>>>>>>>>>>>>> store is still on the same
> >>>>>>>>>>> instance
> >>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
> >>>>>>>>>>>>>>>>>>>>> changes to
> >>>>>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> store was migrated away
> >>>>>>>>>>>>>>>>>>>>> during rebalancing).
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The KIP already updated,
> >>>>>>>>>>>>>>>>>>>> please take a look. :)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48
> >>>>>>>>>>>>>>>>>>>> PM Matthias J. Sax <
> >>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Any update on this KIP?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM,
> >>>>>>>>>>>>>>>>>>>>> Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply.
> >>>>>>>>>>>>>>>>>>>>>> The 2.4 deadline kept us
> >>> quite
> >>>>>>>>>>> busy.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> Why is it only thrown if the state is REBALANCING? A
> >>>>> store
> >>>>>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>>>>> migrated during a
> >>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
> >>>>>>>>>>>>>>>>>>>>>> Streams might
> >>>>> resume
> >>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>> RUNNING state and
> >>>>>>>>>>>>>>>>>>>>>> afterward somebody tries
> >>>>>>>>>>>>>>>>>>>>>> to use an
> >>> old
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>>>>> Also, if state is
> >>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
> >>>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
> >>>>>>>>>>>>>>>>>>>>>> Hence, I think
> >>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>>>>>> does only make sense
> >>> during
> >>>>>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Why do we need to
> >>>>>>>>>>>>>>>>>>>>>> distinguish between
> >>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Last, why do we
> >>>>>>>>>>>>>>>>>>>>>> distinguish between
> >>>>>>>>>>>>>>>>>>>>>> `KafkaStreams`
> >>>>>>>>>>> instance and
> >>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me,
> >>>>>>>>>>>>>>>>>>>>>> it seems we should
> >>>>>>>>>>>>>>>>>>>>>> always
> >>> refer
> >>>>> to
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>>>>>> because that is the level
> >>>>>>>>>>>>>>>>>>>>>> of granularity in which
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>>>>> IQ
> >>>>>>>>>>>>>>>>>>>>> atm.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Last, for
> >>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
> >>>>>>>>>>>>>>>>>>>>>> I would add
> >>>>> that a
> >>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>> rediscover the store and
> >>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>> store
> >>>>>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>>>>>> invalid and a new store
> >>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
> >>>>>>>>>>>>>>>>>>>>>> That
> >>>>> is a
> >>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
> >>>>>>>>>>>>>>>>>>>>>> that allows for
> >>>>>>>>>>> "blind"
> >>>>>>>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>>>>>> that either resolve (if
> >>>>>>>>>>>>>>>>>>>>>> the store is still on the
> >>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>> instance
> >>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
> >>>>>>>>>>>>>>>>>>>>>> changes to
> >>>>>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> store was migrated away
> >>>>>>>>>>>>>>>>>>>>>> during rebalancing).
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito
> >>>>>>>>>>>>>>>>>>>>>> Jeng wrote:
> >>>>>>>>>>>>>>>>>>>>>>> My bad. The short link
> >>>>>>>>>>>>>>>>>>>>>>> `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >>>
> >>>>>>>>>>>>>>>>>>>>>>>
> <https://shorturl.at/CDNT9>
> >>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Please use the
> >>>>>>>>>>>>>>>>>>>>>>> following instead:
> >>>>>>>>>>> https://shorturl.at/bkKQU
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at
> >>>>>>>>>>>>>>>>>>>>>>> 10:53 AM Vito Jeng <
> >>>>>>>>>>>>>> vito@is-land.com.tw>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> Thank you for explanation. I agree with your
> >>> opinion.
> >>>>>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()`
> >>>>>>>>>>>>>>>>>>>>>>>> would never throw
> >>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> For the case that corresponding thread crashes
> >>> after we
> >>>>>>>>>>>>>> handed out
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> store handle. We may
> >>>>>>>>>>>>>>>>>>>>>>>> throw
> >>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>> when stream
> >>> thread is
> >>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> running(
> >>>>>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9)
> >>>>>>>>>>>>>>>>>>>>>>>> or throw
> >>>>>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>> store is
> >>>>>>>>>>>>>>>>>>>>>>>> closed(https://shorturl.at/hrvAN).
> >>>>>>>>>>>>>>>>>>>>>>>> So I
> >>> think
> >>>>>>>>>>> we
> >>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> add a new type for
> >>>>>>>>>>>>>>>>>>>>>>>> this case. Does that
> >>>>>>>>>>>>>>>>>>>>>>>> make sense?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>>>>>>>> vs
> >>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> I understand your point. I rename
> >>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> About check unknown state store names:
> >>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
> >>>>>>>>>>>>>>>>>>>>>>>> hint. I add a new
> >>>>>>>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>> `UnknownStateStoreException`
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> this case.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Also, we should
> >>>>>>>>>>>>>>>>>>>>>>>>> still have fatal
> >>>>>>>>>>>>>>>>>>>>>>>>> exception
> >>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>>>>>> Not sure why you
> >>>>>>>>>>> remove
> >>>>>>>>>>>>>> it?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thank you point this,
> >>>>>>>>>>>>>>>>>>>>>>>> already add it
> >>>>>>>>>>>>>>>>>>>>>>>> again.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> The KIP already
> >>>>>>>>>>>>>>>>>>>>>>>> updated, please take
> >>>>>>>>>>>>>>>>>>>>>>>> a look.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> --- Vito
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>
> >>>
> >
> -----BEGIN PGP SIGNATURE-----
>
> iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl45vCwACgkQO4miYXKq
> /OhT4Q//elvNj2BRIMd7SqyQhpa6sSq4k2iG3wFlr/B2xkmkZPWUICt3SpCisIxG
> RlP5ml7Mi8IcWv9jmMux9C1NpftboLudxIUcun/I4cP0T3S7ytXfzdO+V1WuD9xe
> hfeqMwJTytusEv5VO3MyY+jYGqmPJrPJRViZ2Uwsj3Ojk0SZRb/m+b6ah6BGMh0E
> nlUIux7cabg1OZ/ee0x21hcAB32lVIRfPLKZeHCTHuYmlW76X4pXhFxOm69Pumtj
> ciVMr/pl7B+XiMk0C2Po04zQcP8+5/O7LYU4e8ha12NtuNuDTlQLGUb+S9Qrbxb2
> xdFrlC81RROmIdIOTLiJeeBqhzd19llDgMhBf4spJMOeLn7359PX6r/9tDqonJrQ
> wpSp4S54MDYfTPHUPqA6u/FsE0BX4EMU71ckC9rXVRRUnQ8A8cMk/6qWlRu8NvxQ
> IiYuSo48UENlcTHxXC/rzIDp54gvtv2iml8QcOg6tS+hzhAI4yoY+1w4pnjPmRo0
> 4D1hnq5bd1SLrGSGcCxVbN0jtwgkcr50HBb1UkIDRndqapfKwZOMV65tIwTvxaaZ
> r2QPYyPd6ZDgeMdy0r94z0SMwatqGCmJD9EQmSmLBulemrvYxPGUevQls+WgE4WE
> zc3LtoyfdlXlkttTU0Q/dm2H9OU4s3QO+6PEdC+Qwi3aCKKhdUQ=
> =4EMC
> -----END PGP SIGNATURE-----
>
>
>
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Navinder Brar <na...@yahoo.com.INVALID>.
Thanks Vito, for incorporating this. Makes sense.

-Navinder


On Wednesday, February 5, 2020, 12:17 AM, Matthias J. Sax <mj...@apache.org> wrote:

-----BEGIN PGP SIGNED MESSAGE-----
Hash: SHA512

Thanks Vito!

That makes sense to me.


On 2/1/20 11:29 PM, Vito Jeng wrote:
> Hi, folks,
>
> KIP-562(KAFKA-9445) already merged three days ago.
>
> I have updated KIP-216 to reflect the KIP-562. The main change is
> to introduce a new exception `InvalidStateStorePartitionException`,
> will be thrown when user requested partition not available.
>
> Please take a look and any feedback is welcome. Thanks Matthias for
> the reminder.
>
> --- Vito
>
>
> On Thu, Jan 23, 2020 at 2:13 PM Vito Jeng <vi...@is-land.com.tw>
> wrote:
>
>> Got it, thanks Matthias.
>>
>> --- Vito
>>
>>
>> On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax
>> <ma...@confluent.io> wrote:
>>
>>> Thanks Vito.
>>>
>>> I am also ok with either name. Just a personal slight
>>> preference, but not a important.
>>>
>>>
>>> -Matthias
>>>
>>> On 1/21/20 6:52 PM, Vito Jeng wrote:
>>>> Thanks Matthias.
>>>>
>>>> The KIP is about InvalidStateStoreException. I pick
>>>> `StateStoreNotAvailableException` because it may be more
>>> intuitive
>>>> than `StreamsNotRunningException`.
>>>>
>>>> No matter which one picked, it's good to me.
>>>>
>>>> --- Vito
>>>>
>>>>
>>>> On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax
>>>> <ma...@confluent.io> wrote:
>>>>
>>>>> Thanks for updating the KIP!
>>>>>
>>>>> One last comment/question: you kept
>>>>> `StateStoreNotAvailableException`
>>> in
>>>>> favor of `StreamsNotRunningException` (to merge both as
>>>>> suggested).
>>>>>
>>>>> I am wondering, if it might be better to keep
>>>>> `StreamsNotRunningException` instead of
>>>>> `StateStoreNotAvailableException`, because this exception
>>>>> is thrown if Streams is in state PENDING_SHUTDOWN /
>>>>> NOT_RUNNING / ERROR ?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 1/17/20 9:56 PM, John Roesler wrote:
>>>>>> Thanks, Vito. I've just cast my vote. -John
>>>>>>
>>>>>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
>>>>>>> Hi, folks,
>>>>>>>
>>>>>>> Just update the KIP, please take a look.
>>>>>>>
>>>>>>> Thanks!
>>>>>>>
>>>>>>> --- Vito
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng
>>>>>>> <vi...@is-land.com.tw>
>>> wrote:
>>>>>>>
>>>>>>>> Thanks Bill, John and Matthias. Glad you guys joined
>>>>>>>> this
>>> discussion.
>>>>>>>> I got a lot out of the discussion.
>>>>>>>>
>>>>>>>> I would like to update KIP-216 base on John's
>>>>>>>> suggestion to remove
>>> the
>>>>>>>> category.
>>>>>>>>
>>>>>>>>
>>>>>>>> --- Vito
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
>>> matthias@confluent.io
>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>>> Nevertheless, if we omit the categorization, it’s
>>>>>>>>>> moot.
>>>>>>>>>
>>>>>>>>> Ack.
>>>>>>>>>
>>>>>>>>> I am fine to remove the middle tier. As John
>>>>>>>>> pointed out, it might
>>> be
>>>>>>>>> weird to have only one concrete exception type per
>>>>>>>>> category. We can
>>>>> also
>>>>>>>>> explain in detail how to handle each exception in
>>>>>>>>> their JavaDocs.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
>>>>>>>>>> Vito,
>>>>>>>>>>
>>>>>>>>>> Thanks for the updates, the KIP LGTM.
>>>>>>>>>>
>>>>>>>>>> -Bill
>>>>>>>>>>
>>>>>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <
>>> vvcephei@apache.org>
>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Vito,
>>>>>>>>>>>
>>>>>>>>>>> Haha, your archive game is on point!
>>>>>>>>>>>
>>>>>>>>>>> What Matthias said in that email is essentially
>>>>>>>>>>> what I figured
>>> was
>>>>> the
>>>>>>>>>>> rationale. It makes sense, but the point I was
>>>>>>>>>>> making is that
>>> this
>>>>>>>>> really
>>>>>>>>>>> doesn’t seem like a good way to structure a
>>>>>>>>>>> production app. On
>>> the
>>>>>>>>> other
>>>>>>>>>>> hand, considering the exception fatal has a
>>>>>>>>>>> good chance of
>>> avoiding
>>>>> a
>>>>>>>>>>> frustrating debug session if you just forgot to
>>>>>>>>>>> call start.
>>>>>>>>>>>
>>>>>>>>>>> Nevertheless, if we omit the categorization,
>>>>>>>>>>> it’s moot.
>>>>>>>>>>>
>>>>>>>>>>> It would be easy to add a categorization layer
>>>>>>>>>>> later if we want
>>> it,
>>>>> but
>>>>>>>>>>> not very easy to change it if we get it wrong.
>>>>>>>>>>>
>>>>>>>>>>> Thanks for your consideration! -John
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng
>>>>>>>>>>> wrote:
>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>
>>>>>>>>>>>> About `StreamsNotStartedException is strange`
>>>>>>>>>>>> -- The original idea came from Matthias, two
>>>>>>>>>>>> years ago. :) You can reference here:
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>
>>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6
c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>
About omitting the categorization --
>>>>>>>>>>>> It looks reasonable. I'm fine with omitting
>>>>>>>>>>>> the categorization
>>> but
>>>>> not
>>>>>>>>>>> very
>>>>>>>>>>>> sure it is a good choice. Does any other
>>>>>>>>>>>> folks provide opinion?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Hi, folks,
>>>>>>>>>>>>
>>>>>>>>>>>> Just update the KIP-216, please take a look.
>>>>>>>>>>>>
>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng
>>>>>>>>>>>> <vi...@is-land.com.tw>
>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hi, folks,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thank you suggestion, really appreciate it.
>>>>>>>>>>>>> :) I understand your concern. I'll merge
>>> StreamsNotRunningException
>>>>> and
>>>>>>>>>>>>> StateStoreNotAvailableException.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John
>>>>>>>>>>>>> Roesler <
>>> vvcephei@apache.org
>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hey Vito,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, thanks for the KIP. Sorry the
>>>>>>>>>>>>>> discussion has been so
>>> long.
>>>>>>>>>>>>>> Hopefully, we can close it out soon.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I agree we can drop
>>>>>>>>>>>>>> StreamsNotRunningException in favor of
>>>>>>>>>>>>>> just StateStoreNotAvailableException.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Unfortunately, I have some higher-level
>>>>>>>>>>>>>> concerns. The value of these exceptions
>>>>>>>>>>>>>> is that they tell you how to handle the
>>>>>>>>>>>>>> various situations that can arise while
>>>>>>>>>>>>>> querying a distributed data store.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ideally, as a caller, I should be able to
>>>>>>>>>>>>>> just catch
>>> "retriable"
>>>>> or
>>>>>>>>>>>>>> "fatal" and handle them appropriately.
>>>>>>>>>>>>>> Otherwise, there's no point in having
>>>>>>>>>>>>>> categories, and we should just have all
>>>>>>>>>>>>>> the exceptions extend
>>>>>>>>>>>>>> InvalidStateStoreException.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Presently, it's not possible to tell from
>>>>>>>>>>>>>> just the "retriable"/"fatal" distinction
>>>>>>>>>>>>>> what to do. You  can tell from the
>>>>>>>>>>>>>> descriptions of the various exceptions.
>>>>>>>>>>>>>> E.g.:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Retriable: * StreamsRebalancingException:
>>>>>>>>>>>>>> the exact same call should just be
>>>>>>>>>>>>>> retried until the rebalance is complete *
>>>>>>>>>>>>>> StateStoreMigratedException: the store
>>>>>>>>>>>>>> handle is now invalid, so you need to
>>>>>>>>>>>>>> re-discover the instance and get a new
>>>>>>>>>>>>>> handle on that instance. In other words,
>>>>>>>>>>>>>> the query itself may be valid, but the
>>>>>>>>>>>>>> particular method invocation on this
>>>>>>>>>>>>>> particular instance has encountered a
>>>>>>>>>>>>>> fatal exception.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Fatal: * UnknownStateStoreException: this
>>>>>>>>>>>>>> is truly fatal. No amount of retrying or
>>>>>>>>>>>>>> re-discovering is going to get you a
>>>>>>>>>>>>>> handle
>>>>> on a
>>>>>>>>>>>>>> store that doesn't exist in the cluster.
>>>>>>>>>>>>>> * StateStoreNotAvailableException: this
>>>>>>>>>>>>>> is actually
>>> recoverable,
>>>>>>>>>>>>>> since the store might exist in the
>>>>>>>>>>>>>> cluster, but isn't
>>>>> available
>>>>>>>>> on
>>>>>>>>>>>>>> this particular instance (which is shut
>>>>>>>>>>>>>> down or whatever).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Personally, I'm not a fan of code
>>>>>>>>>>>>>> bureaucracy, so I'm 100%
>>> fine
>>>>>>>>>>>>>> with omitting the categorization and just
>>>>>>>>>>>>>> having 5 subclasses of
>>>>>>>>>>>>>> InvalidStateStoreException. Each of them
>>>>>>>>>>>>>> would tell you how to handle them, and
>>>>>>>>>>>>>> it's not too many to really understand
>>>>>>>>>>>>>> and handle each one.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If you really want to have a middle tier,
>>>>>>>>>>>>>> I'd recommend: *
>>>>>>>>>>>>>> RetryableStateStoreException: the exact
>>>>>>>>>>>>>> same call should be repeated. *
>>>>>>>>>>>>>> RecoverableStateStoreException: the store
>>>>>>>>>>>>>> handle should be discarded and the caller
>>>>>>>>>>>>>> should re-discover the location of the
>>>>>>>>>>>>>> store and repeat the query on the correct
>>>>>>>>>>>>>> instance. * FatalStateStoreException: the
>>>>>>>>>>>>>> query/request is totally invalid and will
>>>>>>>>>>>>>> never succeed.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> However, attempting to categorize the
>>>>>>>>>>>>>> proposed exceptions reveals even problems
>>>>>>>>>>>>>> with this categorization: Retriable: *
>>>>>>>>>>>>>> StreamsRebalancingException Recoverable:
>>>>>>>>>>>>>> * StateStoreMigratedException *
>>>>>>>>>>>>>> StreamsNotRunningException Fatal: *
>>>>>>>>>>>>>> UnknownStateStoreException
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> But StreamsNotStartedException is
>>>>>>>>>>>>>> strange... It means that one code path
>>>>>>>>>>>>>> got a handle on a specific KafkaStreams
>>>>>>>>>>>>>> object instance and sent it a query
>>>>>>>>>>>>>> before another code path invoked the
>>>>>>>>>>>>>> start() method on the exact same object
>>>>>>>>>>>>>> instance. It seems like the most likely
>>>>>>>>>>>>>> scenario is that whoever wrote the
>>>>>>>>>>>>>> program just forgot to call start()
>>>>>>>>>>>>>> before querying, in which case, retrying
>>>>>>>>>>>>>> isn't going to help, and a fatal
>>> exception
>>>>>>>>>>>>>> is more appropriate. I.e., it sounds like
>>>>>>>>>>>>>> a "first 15 minutes experience" problem,
>>>>>>>>>>>>>> and making it fatal would be more
>>>>>>>>>>>>>> helpful. Even in a production context,
>>>>>>>>>>>>>> there's no reason not to sequence your
>>>>>>>>>>>>>> application startup such that you don't
>>>>>>>>>>>>>> accept queries until after Streams is
>>>>>>>>>>>>>> started. Thus, I guess I'd categorize it
>>>>>>>>>>>>>> under "fatal".
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regardless of whether you make it fatal
>>>>>>>>>>>>>> or retriable, you'd still have a whole
>>>>>>>>>>>>>> category with only one exception in it,
>>>>>>>>>>>>>> and the other two categories only have
>>>>>>>>>>>>>> two exceptions. Plus, as you pointed out
>>>>>>>>>>>>>> in the KIP, you can't get all exceptions
>>>>>>>>>>>>>> in all cases anyway: * store() can only
>>>>>>>>>>>>>> throw NotStarted, NotRunning, and
>>>>>>>>>>>>>> Unknown * actual store queries can only
>>>>>>>>>>>>>> throw Rebalancing, Migrated, and
>>>>>>>>>>>>>> NotRunning
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thus, in practice also, there are exactly
>>>>>>>>>>>>>> three categories and also exactly three
>>>>>>>>>>>>>> exception types. It doesn't seem like
>>>>>>>>>>>>>> there's a great advantage to the
>>>>>>>>>>>>>> categories here. To avoid the
>>>>>>>>>>>>>> categorization problem and also to
>>>>>>>>>>>>>> clarify what exceptions can actually be
>>>>>>>>>>>>>> thrown in different circumstances, it
>>>>>>>>>>>>>> seems like we should just: * get rid of
>>>>>>>>>>>>>> the middle tier and make all the
>>>>>>>>>>>>>> exceptions extend
>>>>>>>>>>>>>> InvalidStateStoreException * drop
>>>>>>>>>>>>>> StateStoreNotAvailableException in favor
>>>>>>>>>>>>>> of StreamsNotRunningException * clearly
>>>>>>>>>>>>>> document on all public methods which
>>>>>>>>>>>>>> exceptions need to be handled
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> How do you feel about this? Thanks,
>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill
>>>>>>>>>>>>>> Bejeck wrote:
>>>>>>>>>>>>>>> Thanks for KIP Vito.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Overall the KIP LGTM, but I'd have to
>>>>>>>>>>>>>>> agree with others on
>>>>> merging
>>>>>>>>>>> the
>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>> classes.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Since in both cases, the thread state
>>>>>>>>>>>>>>> is in
>>> `PENDING_SHUTDOWN ||
>>>>>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure
>>>>>>>>>>>>>>> how we could
>>> distinguish
>>>>>>>>>>> when to
>>>>>>>>>>>>>>> use the different exceptions.  Maybe a
>>>>>>>>>>>>>>> good middle ground would be to have a
>>>>> detailed
>>>>>>>>>>>>>>> exception message.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The KIP freeze is close, so I think if
>>>>>>>>>>>>>>> we can agree on this,
>>> we
>>>>> can
>>>>>>>>>>>>>> wrap up
>>>>>>>>>>>>>>> the voting soon.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks, Bill
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM
>>>>>>>>>>>>>>> Matthias J. Sax <
>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Vito,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It's still unclear to me what the
>>>>>>>>>>>>>>>> advantage is, to have both
>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For both cased, the state is
>>>>>>>>>>>>>>>> `PENDING_SHUTDOWN /
>>> NOT_RUNNING /
>>>>>>>>>>> ERROR`
>>>>>>>>>>>>>>>> and thus, for a user point of view,
>>>>>>>>>>>>>>>> why does it matter if
>>> the
>>>>>>>>>>> store is
>>>>>>>>>>>>>>>> closed on not? I don't understand
>>>>>>>>>>>>>>>> why/how this information
>>>>> would
>>>>>>>>>>> be
>>>>>>>>>>>>>>>> useful? Do you have a concrete
>>>>>>>>>>>>>>>> example in mind how a user
>>> would
>>>>>>>>>>> react
>>>>>>>>>>>>>>>> differently to both exceptions?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> @Vinoth: about
>>>>>>>>>>>>>>>> `StreamsRebalancingException` -- to
>>>>>>>>>>>>>>>> me, it
>>> seems
>>>>>>>>>>> best
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> actually do this on a per-query
>>>>>>>>>>>>>>>> basis, ie, have an overload
>>>>>>>>>>>>>>>> `KafkaStreams#store(...)` that takes
>>>>>>>>>>>>>>>> a boolean flag that
>>> allow
>>>>> to
>>>>>>>>>>>>>>>> _disable_ the exception and opt-in to
>>>>>>>>>>>>>>>> query a active store
>>>>> during
>>>>>>>>>>>>>>>> recovery. However, as KIP-535
>>>>>>>>>>>>>>>> actually introduces this
>>> change
>>>>> in
>>>>>>>>>>>>>>>> behavior, I think KIP-216 should not
>>>>>>>>>>>>>>>> cover this, but KIP-535
>>>>>>>>>>> should be
>>>>>>>>>>>>>>>> updated. I'll follow up on the other
>>>>>>>>>>>>>>>> KIP thread to raise
>>> this
>>>>>>>>>>> point.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> Hi, Matthias & Vinoth,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> What is still unclear to me is,
>>>>>>>>>>>>>>>>>> what we gain by having
>>> both
>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>> `StateStoreNotAvailableException`. Both
>>>>>>>>>>>>>>>>>> exception are thrown when
>>>>>>>>>>>>>>>>>> KafkaStreams is in state
>>>>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a
>>>>>>>>>>>>>>>>>> user what do I gain to
>>> know
>>>>>>>>>>> if the
>>>>>>>>>>>>>>>>>> state store is closed on not -- I
>>>>>>>>>>>>>>>>>> can't query it anyway?
>>>>> Maybe
>>>>>>>>>>> I
>>>>>>>>>>>>>> miss
>>>>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Yes, both
>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>>>>> are fatal exception. But
>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>>>>> is fatal exception
>>> about
>>>>>>>>>>> state
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>> related. I think it would be
>>>>>>>>>>>>>>>>> helpful that if user need to
>>> distinguish
>>>>>>>>>>> these
>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>> different case to handle it.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'm not very sure, does that make
>>>>>>>>>>>>>>>>> sense?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM
>>>>>>>>>>>>>>>>> Vinoth Chandar <
>>>>>>>>>>> vinoth@apache.org>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> +1 on merging
>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`,
>>>>>>>>>>>>>>>>>> both exceptions are
>>> fatal
>>>>>>>>>>>>>> anyway. IMO
>>>>>>>>>>>>>>>>>> its best to have these exceptions
>>>>>>>>>>>>>>>>>> be about the state store
>>>>>>>>>>> (and not
>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>> state), to easier understanding.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Additionally, KIP-535 allows for
>>>>>>>>>>>>>>>>>> querying of state stores
>>> in
>>>>>>>>>>>>>> rebalancing
>>>>>>>>>>>>>>>>>> state. So do we need the
>>>>>>>>>>>>>>>>>> StreamsRebalancingException?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias
>>>>>>>>>>>>>>>>>> J. Sax" <
>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> Sorry that I dropped the ball
>>>>>>>>>>>>>>>>>>> on this...
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP.
>>>>>>>>>>>>>>>>>>> Overall LGTM now. Feel free
>>> to
>>>>>>>>>>> start
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> VOTE
>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> What is still unclear to me is,
>>>>>>>>>>>>>>>>>>> what we gain by having
>>> both
>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException`
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> `StateStoreNotAvailableException`.
>>>>>>>>>>>>>>>> Both
>>>>>>>>>>>>>>>>>>> exception are thrown when
>>>>>>>>>>>>>>>>>>> KafkaStreams is in state
>>>>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as
>>>>>>>>>>>>>>>>>>> a user what do I gain to
>>> know
>>>>>>>>>>> if
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> state store is closed on not --
>>>>>>>>>>>>>>>>>>> I can't query it anyway?
>>>>>>>>>>> Maybe I
>>>>>>>>>>>>>> miss
>>>>>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> Sorry for the late reply,
>>>>>>>>>>>>>>>>>>>> thanks for the review.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
Why is it only thrown if the state is REBALANCING? A
>>> store
>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>> migrated during a
>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
>>>>>>>>>>>>>>>>>>>>> Streams might
>>>>> resume
>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>> RUNNING state and afterward
>>>>>>>>>>>>>>>>>>>>> somebody tries to use an
>>> old
>>>>>>>>>>> store
>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>>>> Also, if state is
>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
>>>>>>>>>>>>>>>>>>>>> Hence, I think
>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>>>>> does only make sense
>>> during
>>>>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thank you point this, already
>>>>>>>>>>>>>>>>>>>> updated.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Why do we need to distinguish
>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
`KafkaStreamsNotRunningException` may be caused by
>>> various
>>>>>>>>>>>>>> reasons, I
>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>> it would be helpful that the
>>>>>>>>>>>>>>>>>>>> user can distinguish whether
>>>>>>>>>>>>>>>>>>>> it is caused by the state
>>>>> store
>>>>>>>>>>>>>> closed.
>>>>>>>>>>>>>>>>>>>> (Maybe I am wrong...)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Last, why do we distinguish
>>>>>>>>>>>>>>>>>>>> between `KafkaStreams`
>>> instance
>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it
>>>>>>>>>>>>>>>>>>>>> seems we should always
>>> refer to
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>>>>> because that is the level
>>>>>>>>>>>>>>>>>>>>> of granularity in which we
>>>>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>>>>> IQ atm.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Totally agree. Do you mean
>>>>>>>>>>>>>>>>>>>> the naming of state store
>>>>>>>>>>> exceptions?
>>>>>>>>>>>>>>>>>>>> I don't have special reason
>>>>>>>>>>>>>>>>>>>> to distinguish these two.
>>>>>>>>>>>>>>>>>>>> Your suggestion look more
>>>>>>>>>>>>>>>>>>>> reasonable for the exception
>>>>>>>>>>> naming.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Last, for
>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
>>>>>>>>>>>>>>>>>>>> I would add
>>> that a
>>>>>>>>>>> user
>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> rediscover the store and
>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
>>>>>>>>>>>>>>>>>>>>> the
>>> store
>>>>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>>>>> invalid and a new store
>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
>>>>>>>>>>>>>>>>>>>>> That
>>> is
>>>>> a
>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
>>>>>>>>>>>>>>>>>>>>> that allows for
>>>>>>>>>>> "blind"
>>>>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>>>>> that either resolve (if the
>>>>>>>>>>>>>>>>>>>>> store is still on the same
>>>>>>>>>>> instance
>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
>>>>>>>>>>>>>>>>>>>>> changes to
>>>>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> store was migrated away
>>>>>>>>>>>>>>>>>>>>> during rebalancing).
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The KIP already updated,
>>>>>>>>>>>>>>>>>>>> please take a look. :)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48
>>>>>>>>>>>>>>>>>>>> PM Matthias J. Sax <
>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Any update on this KIP?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM,
>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply.
>>>>>>>>>>>>>>>>>>>>>> The 2.4 deadline kept us
>>> quite
>>>>>>>>>>> busy.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
Why is it only thrown if the state is REBALANCING? A
>>>>> store
>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>>> migrated during a
>>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
>>>>>>>>>>>>>>>>>>>>>> Streams might
>>>>> resume
>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>> RUNNING state and
>>>>>>>>>>>>>>>>>>>>>> afterward somebody tries
>>>>>>>>>>>>>>>>>>>>>> to use an
>>> old
>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>>>>> Also, if state is
>>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
>>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
>>>>>>>>>>>>>>>>>>>>>> Hence, I think
>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>>>>>> does only make sense
>>> during
>>>>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Why do we need to
>>>>>>>>>>>>>>>>>>>>>> distinguish between
>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Last, why do we
>>>>>>>>>>>>>>>>>>>>>> distinguish between
>>>>>>>>>>>>>>>>>>>>>> `KafkaStreams`
>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me,
>>>>>>>>>>>>>>>>>>>>>> it seems we should
>>>>>>>>>>>>>>>>>>>>>> always
>>> refer
>>>>> to
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>>>>>> because that is the level
>>>>>>>>>>>>>>>>>>>>>> of granularity in which
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>>>>> IQ
>>>>>>>>>>>>>>>>>>>>> atm.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Last, for
>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
>>>>>>>>>>>>>>>>>>>>>> I would add
>>>>> that a
>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>> rediscover the store and
>>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
>>>>>>>>>>>>>>>>>>>>>> the
>>>>> store
>>>>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>>>>>> invalid and a new store
>>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
>>>>>>>>>>>>>>>>>>>>>> That
>>>>> is a
>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
>>>>>>>>>>>>>>>>>>>>>> that allows for
>>>>>>>>>>> "blind"
>>>>>>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>>>>>> that either resolve (if
>>>>>>>>>>>>>>>>>>>>>> the store is still on the
>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>> instance
>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
>>>>>>>>>>>>>>>>>>>>>> changes to
>>>>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> store was migrated away
>>>>>>>>>>>>>>>>>>>>>> during rebalancing).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito
>>>>>>>>>>>>>>>>>>>>>> Jeng wrote:
>>>>>>>>>>>>>>>>>>>>>>> My bad. The short link
>>>>>>>>>>>>>>>>>>>>>>> `https://shorturl.at/CDNT9`
>>>
>>>>>>>>>>>>>>>>>>>>>>>
<https://shorturl.at/CDNT9>
>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
<https://shorturl.at/CDNT9> seems incorrect.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Please use the
>>>>>>>>>>>>>>>>>>>>>>> following instead:
>>>>>>>>>>> https://shorturl.at/bkKQU
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at
>>>>>>>>>>>>>>>>>>>>>>> 10:53 AM Vito Jeng <
>>>>>>>>>>>>>> vito@is-land.com.tw>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
Thank you for explanation. I agree with your
>>> opinion.
>>>>>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()`
>>>>>>>>>>>>>>>>>>>>>>>> would never throw
>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
For the case that corresponding thread crashes
>>> after we
>>>>>>>>>>>>>> handed out
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> store handle. We may
>>>>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
In `StreamThreadStateStoreProvider`, we would throw
>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>> when stream
>>> thread is
>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> running(
>>>>>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9)
>>>>>>>>>>>>>>>>>>>>>>>> or throw
>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>>>>>>> closed(https://shorturl.at/hrvAN).
>>>>>>>>>>>>>>>>>>>>>>>> So I
>>> think
>>>>>>>>>>> we
>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> add a new type for
>>>>>>>>>>>>>>>>>>>>>>>> this case. Does that
>>>>>>>>>>>>>>>>>>>>>>>> make sense?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>>> vs
>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
I understand your point. I rename
>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
About check unknown state store names:
>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
>>>>>>>>>>>>>>>>>>>>>>>> hint. I add a new
>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>> `UnknownStateStoreException`
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> this case.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Also, we should
>>>>>>>>>>>>>>>>>>>>>>>>> still have fatal
>>>>>>>>>>>>>>>>>>>>>>>>> exception
>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>>>> Not sure why you
>>>>>>>>>>> remove
>>>>>>>>>>>>>> it?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thank you point this,
>>>>>>>>>>>>>>>>>>>>>>>> already add it
>>>>>>>>>>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> The KIP already
>>>>>>>>>>>>>>>>>>>>>>>> updated, please take
>>>>>>>>>>>>>>>>>>>>>>>> a look.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>>
>
-----BEGIN PGP SIGNATURE-----

iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl45vCwACgkQO4miYXKq
/OhT4Q//elvNj2BRIMd7SqyQhpa6sSq4k2iG3wFlr/B2xkmkZPWUICt3SpCisIxG
RlP5ml7Mi8IcWv9jmMux9C1NpftboLudxIUcun/I4cP0T3S7ytXfzdO+V1WuD9xe
hfeqMwJTytusEv5VO3MyY+jYGqmPJrPJRViZ2Uwsj3Ojk0SZRb/m+b6ah6BGMh0E
nlUIux7cabg1OZ/ee0x21hcAB32lVIRfPLKZeHCTHuYmlW76X4pXhFxOm69Pumtj
ciVMr/pl7B+XiMk0C2Po04zQcP8+5/O7LYU4e8ha12NtuNuDTlQLGUb+S9Qrbxb2
xdFrlC81RROmIdIOTLiJeeBqhzd19llDgMhBf4spJMOeLn7359PX6r/9tDqonJrQ
wpSp4S54MDYfTPHUPqA6u/FsE0BX4EMU71ckC9rXVRRUnQ8A8cMk/6qWlRu8NvxQ
IiYuSo48UENlcTHxXC/rzIDp54gvtv2iml8QcOg6tS+hzhAI4yoY+1w4pnjPmRo0
4D1hnq5bd1SLrGSGcCxVbN0jtwgkcr50HBb1UkIDRndqapfKwZOMV65tIwTvxaaZ
r2QPYyPd6ZDgeMdy0r94z0SMwatqGCmJD9EQmSmLBulemrvYxPGUevQls+WgE4WE
zc3LtoyfdlXlkttTU0Q/dm2H9OU4s3QO+6PEdC+Qwi3aCKKhdUQ=
=4EMC
-----END PGP SIGNATURE-----





Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <mj...@apache.org>.
-----BEGIN PGP SIGNED MESSAGE-----
Hash: SHA512

Thanks Vito!

That makes sense to me.


On 2/1/20 11:29 PM, Vito Jeng wrote:
> Hi, folks,
>
> KIP-562(KAFKA-9445) already merged three days ago.
>
> I have updated KIP-216 to reflect the KIP-562. The main change is
> to introduce a new exception `InvalidStateStorePartitionException`,
> will be thrown when user requested partition not available.
>
> Please take a look and any feedback is welcome. Thanks Matthias for
> the reminder.
>
> --- Vito
>
>
> On Thu, Jan 23, 2020 at 2:13 PM Vito Jeng <vi...@is-land.com.tw>
> wrote:
>
>> Got it, thanks Matthias.
>>
>> --- Vito
>>
>>
>> On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax
>> <ma...@confluent.io> wrote:
>>
>>> Thanks Vito.
>>>
>>> I am also ok with either name. Just a personal slight
>>> preference, but not a important.
>>>
>>>
>>> -Matthias
>>>
>>> On 1/21/20 6:52 PM, Vito Jeng wrote:
>>>> Thanks Matthias.
>>>>
>>>> The KIP is about InvalidStateStoreException. I pick
>>>> `StateStoreNotAvailableException` because it may be more
>>> intuitive
>>>> than `StreamsNotRunningException`.
>>>>
>>>> No matter which one picked, it's good to me.
>>>>
>>>> --- Vito
>>>>
>>>>
>>>> On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax
>>>> <ma...@confluent.io> wrote:
>>>>
>>>>> Thanks for updating the KIP!
>>>>>
>>>>> One last comment/question: you kept
>>>>> `StateStoreNotAvailableException`
>>> in
>>>>> favor of `StreamsNotRunningException` (to merge both as
>>>>> suggested).
>>>>>
>>>>> I am wondering, if it might be better to keep
>>>>> `StreamsNotRunningException` instead of
>>>>> `StateStoreNotAvailableException`, because this exception
>>>>> is thrown if Streams is in state PENDING_SHUTDOWN /
>>>>> NOT_RUNNING / ERROR ?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 1/17/20 9:56 PM, John Roesler wrote:
>>>>>> Thanks, Vito. I've just cast my vote. -John
>>>>>>
>>>>>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
>>>>>>> Hi, folks,
>>>>>>>
>>>>>>> Just update the KIP, please take a look.
>>>>>>>
>>>>>>> Thanks!
>>>>>>>
>>>>>>> --- Vito
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng
>>>>>>> <vi...@is-land.com.tw>
>>> wrote:
>>>>>>>
>>>>>>>> Thanks Bill, John and Matthias. Glad you guys joined
>>>>>>>> this
>>> discussion.
>>>>>>>> I got a lot out of the discussion.
>>>>>>>>
>>>>>>>> I would like to update KIP-216 base on John's
>>>>>>>> suggestion to remove
>>> the
>>>>>>>> category.
>>>>>>>>
>>>>>>>>
>>>>>>>> --- Vito
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
>>> matthias@confluent.io
>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>>> Nevertheless, if we omit the categorization, it’s
>>>>>>>>>> moot.
>>>>>>>>>
>>>>>>>>> Ack.
>>>>>>>>>
>>>>>>>>> I am fine to remove the middle tier. As John
>>>>>>>>> pointed out, it might
>>> be
>>>>>>>>> weird to have only one concrete exception type per
>>>>>>>>> category. We can
>>>>> also
>>>>>>>>> explain in detail how to handle each exception in
>>>>>>>>> their JavaDocs.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
>>>>>>>>>> Vito,
>>>>>>>>>>
>>>>>>>>>> Thanks for the updates, the KIP LGTM.
>>>>>>>>>>
>>>>>>>>>> -Bill
>>>>>>>>>>
>>>>>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <
>>> vvcephei@apache.org>
>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Vito,
>>>>>>>>>>>
>>>>>>>>>>> Haha, your archive game is on point!
>>>>>>>>>>>
>>>>>>>>>>> What Matthias said in that email is essentially
>>>>>>>>>>> what I figured
>>> was
>>>>> the
>>>>>>>>>>> rationale. It makes sense, but the point I was
>>>>>>>>>>> making is that
>>> this
>>>>>>>>> really
>>>>>>>>>>> doesn’t seem like a good way to structure a
>>>>>>>>>>> production app. On
>>> the
>>>>>>>>> other
>>>>>>>>>>> hand, considering the exception fatal has a
>>>>>>>>>>> good chance of
>>> avoiding
>>>>> a
>>>>>>>>>>> frustrating debug session if you just forgot to
>>>>>>>>>>> call start.
>>>>>>>>>>>
>>>>>>>>>>> Nevertheless, if we omit the categorization,
>>>>>>>>>>> it’s moot.
>>>>>>>>>>>
>>>>>>>>>>> It would be easy to add a categorization layer
>>>>>>>>>>> later if we want
>>> it,
>>>>> but
>>>>>>>>>>> not very easy to change it if we get it wrong.
>>>>>>>>>>>
>>>>>>>>>>> Thanks for your consideration! -John
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng
>>>>>>>>>>> wrote:
>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>
>>>>>>>>>>>> About `StreamsNotStartedException is strange`
>>>>>>>>>>>> -- The original idea came from Matthias, two
>>>>>>>>>>>> years ago. :) You can reference here:
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>
>>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6
c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>
About omitting the categorization --
>>>>>>>>>>>> It looks reasonable. I'm fine with omitting
>>>>>>>>>>>> the categorization
>>> but
>>>>> not
>>>>>>>>>>> very
>>>>>>>>>>>> sure it is a good choice. Does any other
>>>>>>>>>>>> folks provide opinion?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Hi, folks,
>>>>>>>>>>>>
>>>>>>>>>>>> Just update the KIP-216, please take a look.
>>>>>>>>>>>>
>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng
>>>>>>>>>>>> <vi...@is-land.com.tw>
>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hi, folks,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thank you suggestion, really appreciate it.
>>>>>>>>>>>>> :) I understand your concern. I'll merge
>>> StreamsNotRunningException
>>>>> and
>>>>>>>>>>>>> StateStoreNotAvailableException.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John
>>>>>>>>>>>>> Roesler <
>>> vvcephei@apache.org
>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hey Vito,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, thanks for the KIP. Sorry the
>>>>>>>>>>>>>> discussion has been so
>>> long.
>>>>>>>>>>>>>> Hopefully, we can close it out soon.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I agree we can drop
>>>>>>>>>>>>>> StreamsNotRunningException in favor of
>>>>>>>>>>>>>> just StateStoreNotAvailableException.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Unfortunately, I have some higher-level
>>>>>>>>>>>>>> concerns. The value of these exceptions
>>>>>>>>>>>>>> is that they tell you how to handle the
>>>>>>>>>>>>>> various situations that can arise while
>>>>>>>>>>>>>> querying a distributed data store.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ideally, as a caller, I should be able to
>>>>>>>>>>>>>> just catch
>>> "retriable"
>>>>> or
>>>>>>>>>>>>>> "fatal" and handle them appropriately.
>>>>>>>>>>>>>> Otherwise, there's no point in having
>>>>>>>>>>>>>> categories, and we should just have all
>>>>>>>>>>>>>> the exceptions extend
>>>>>>>>>>>>>> InvalidStateStoreException.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Presently, it's not possible to tell from
>>>>>>>>>>>>>> just the "retriable"/"fatal" distinction
>>>>>>>>>>>>>> what to do. You  can tell from the
>>>>>>>>>>>>>> descriptions of the various exceptions.
>>>>>>>>>>>>>> E.g.:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Retriable: * StreamsRebalancingException:
>>>>>>>>>>>>>> the exact same call should just be
>>>>>>>>>>>>>> retried until the rebalance is complete *
>>>>>>>>>>>>>> StateStoreMigratedException: the store
>>>>>>>>>>>>>> handle is now invalid, so you need to
>>>>>>>>>>>>>> re-discover the instance and get a new
>>>>>>>>>>>>>> handle on that instance. In other words,
>>>>>>>>>>>>>> the query itself may be valid, but the
>>>>>>>>>>>>>> particular method invocation on this
>>>>>>>>>>>>>> particular instance has encountered a
>>>>>>>>>>>>>> fatal exception.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Fatal: * UnknownStateStoreException: this
>>>>>>>>>>>>>> is truly fatal. No amount of retrying or
>>>>>>>>>>>>>> re-discovering is going to get you a
>>>>>>>>>>>>>> handle
>>>>> on a
>>>>>>>>>>>>>> store that doesn't exist in the cluster.
>>>>>>>>>>>>>> * StateStoreNotAvailableException: this
>>>>>>>>>>>>>> is actually
>>> recoverable,
>>>>>>>>>>>>>> since the store might exist in the
>>>>>>>>>>>>>> cluster, but isn't
>>>>> available
>>>>>>>>> on
>>>>>>>>>>>>>> this particular instance (which is shut
>>>>>>>>>>>>>> down or whatever).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Personally, I'm not a fan of code
>>>>>>>>>>>>>> bureaucracy, so I'm 100%
>>> fine
>>>>>>>>>>>>>> with omitting the categorization and just
>>>>>>>>>>>>>> having 5 subclasses of
>>>>>>>>>>>>>> InvalidStateStoreException. Each of them
>>>>>>>>>>>>>> would tell you how to handle them, and
>>>>>>>>>>>>>> it's not too many to really understand
>>>>>>>>>>>>>> and handle each one.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If you really want to have a middle tier,
>>>>>>>>>>>>>> I'd recommend: *
>>>>>>>>>>>>>> RetryableStateStoreException: the exact
>>>>>>>>>>>>>> same call should be repeated. *
>>>>>>>>>>>>>> RecoverableStateStoreException: the store
>>>>>>>>>>>>>> handle should be discarded and the caller
>>>>>>>>>>>>>> should re-discover the location of the
>>>>>>>>>>>>>> store and repeat the query on the correct
>>>>>>>>>>>>>> instance. * FatalStateStoreException: the
>>>>>>>>>>>>>> query/request is totally invalid and will
>>>>>>>>>>>>>> never succeed.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> However, attempting to categorize the
>>>>>>>>>>>>>> proposed exceptions reveals even problems
>>>>>>>>>>>>>> with this categorization: Retriable: *
>>>>>>>>>>>>>> StreamsRebalancingException Recoverable:
>>>>>>>>>>>>>> * StateStoreMigratedException *
>>>>>>>>>>>>>> StreamsNotRunningException Fatal: *
>>>>>>>>>>>>>> UnknownStateStoreException
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> But StreamsNotStartedException is
>>>>>>>>>>>>>> strange... It means that one code path
>>>>>>>>>>>>>> got a handle on a specific KafkaStreams
>>>>>>>>>>>>>> object instance and sent it a query
>>>>>>>>>>>>>> before another code path invoked the
>>>>>>>>>>>>>> start() method on the exact same object
>>>>>>>>>>>>>> instance. It seems like the most likely
>>>>>>>>>>>>>> scenario is that whoever wrote the
>>>>>>>>>>>>>> program just forgot to call start()
>>>>>>>>>>>>>> before querying, in which case, retrying
>>>>>>>>>>>>>> isn't going to help, and a fatal
>>> exception
>>>>>>>>>>>>>> is more appropriate. I.e., it sounds like
>>>>>>>>>>>>>> a "first 15 minutes experience" problem,
>>>>>>>>>>>>>> and making it fatal would be more
>>>>>>>>>>>>>> helpful. Even in a production context,
>>>>>>>>>>>>>> there's no reason not to sequence your
>>>>>>>>>>>>>> application startup such that you don't
>>>>>>>>>>>>>> accept queries until after Streams is
>>>>>>>>>>>>>> started. Thus, I guess I'd categorize it
>>>>>>>>>>>>>> under "fatal".
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regardless of whether you make it fatal
>>>>>>>>>>>>>> or retriable, you'd still have a whole
>>>>>>>>>>>>>> category with only one exception in it,
>>>>>>>>>>>>>> and the other two categories only have
>>>>>>>>>>>>>> two exceptions. Plus, as you pointed out
>>>>>>>>>>>>>> in the KIP, you can't get all exceptions
>>>>>>>>>>>>>> in all cases anyway: * store() can only
>>>>>>>>>>>>>> throw NotStarted, NotRunning, and
>>>>>>>>>>>>>> Unknown * actual store queries can only
>>>>>>>>>>>>>> throw Rebalancing, Migrated, and
>>>>>>>>>>>>>> NotRunning
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thus, in practice also, there are exactly
>>>>>>>>>>>>>> three categories and also exactly three
>>>>>>>>>>>>>> exception types. It doesn't seem like
>>>>>>>>>>>>>> there's a great advantage to the
>>>>>>>>>>>>>> categories here. To avoid the
>>>>>>>>>>>>>> categorization problem and also to
>>>>>>>>>>>>>> clarify what exceptions can actually be
>>>>>>>>>>>>>> thrown in different circumstances, it
>>>>>>>>>>>>>> seems like we should just: * get rid of
>>>>>>>>>>>>>> the middle tier and make all the
>>>>>>>>>>>>>> exceptions extend
>>>>>>>>>>>>>> InvalidStateStoreException * drop
>>>>>>>>>>>>>> StateStoreNotAvailableException in favor
>>>>>>>>>>>>>> of StreamsNotRunningException * clearly
>>>>>>>>>>>>>> document on all public methods which
>>>>>>>>>>>>>> exceptions need to be handled
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> How do you feel about this? Thanks,
>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill
>>>>>>>>>>>>>> Bejeck wrote:
>>>>>>>>>>>>>>> Thanks for KIP Vito.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Overall the KIP LGTM, but I'd have to
>>>>>>>>>>>>>>> agree with others on
>>>>> merging
>>>>>>>>>>> the
>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>> classes.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Since in both cases, the thread state
>>>>>>>>>>>>>>> is in
>>> `PENDING_SHUTDOWN ||
>>>>>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure
>>>>>>>>>>>>>>> how we could
>>> distinguish
>>>>>>>>>>> when to
>>>>>>>>>>>>>>> use the different exceptions.  Maybe a
>>>>>>>>>>>>>>> good middle ground would be to have a
>>>>> detailed
>>>>>>>>>>>>>>> exception message.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The KIP freeze is close, so I think if
>>>>>>>>>>>>>>> we can agree on this,
>>> we
>>>>> can
>>>>>>>>>>>>>> wrap up
>>>>>>>>>>>>>>> the voting soon.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks, Bill
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM
>>>>>>>>>>>>>>> Matthias J. Sax <
>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Vito,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It's still unclear to me what the
>>>>>>>>>>>>>>>> advantage is, to have both
>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For both cased, the state is
>>>>>>>>>>>>>>>> `PENDING_SHUTDOWN /
>>> NOT_RUNNING /
>>>>>>>>>>> ERROR`
>>>>>>>>>>>>>>>> and thus, for a user point of view,
>>>>>>>>>>>>>>>> why does it matter if
>>> the
>>>>>>>>>>> store is
>>>>>>>>>>>>>>>> closed on not? I don't understand
>>>>>>>>>>>>>>>> why/how this information
>>>>> would
>>>>>>>>>>> be
>>>>>>>>>>>>>>>> useful? Do you have a concrete
>>>>>>>>>>>>>>>> example in mind how a user
>>> would
>>>>>>>>>>> react
>>>>>>>>>>>>>>>> differently to both exceptions?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> @Vinoth: about
>>>>>>>>>>>>>>>> `StreamsRebalancingException` -- to
>>>>>>>>>>>>>>>> me, it
>>> seems
>>>>>>>>>>> best
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> actually do this on a per-query
>>>>>>>>>>>>>>>> basis, ie, have an overload
>>>>>>>>>>>>>>>> `KafkaStreams#store(...)` that takes
>>>>>>>>>>>>>>>> a boolean flag that
>>> allow
>>>>> to
>>>>>>>>>>>>>>>> _disable_ the exception and opt-in to
>>>>>>>>>>>>>>>> query a active store
>>>>> during
>>>>>>>>>>>>>>>> recovery. However, as KIP-535
>>>>>>>>>>>>>>>> actually introduces this
>>> change
>>>>> in
>>>>>>>>>>>>>>>> behavior, I think KIP-216 should not
>>>>>>>>>>>>>>>> cover this, but KIP-535
>>>>>>>>>>> should be
>>>>>>>>>>>>>>>> updated. I'll follow up on the other
>>>>>>>>>>>>>>>> KIP thread to raise
>>> this
>>>>>>>>>>> point.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> Hi, Matthias & Vinoth,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> What is still unclear to me is,
>>>>>>>>>>>>>>>>>> what we gain by having
>>> both
>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>> `StateStoreNotAvailableException`. Both
>>>>>>>>>>>>>>>>>> exception are thrown when
>>>>>>>>>>>>>>>>>> KafkaStreams is in state
>>>>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a
>>>>>>>>>>>>>>>>>> user what do I gain to
>>> know
>>>>>>>>>>> if the
>>>>>>>>>>>>>>>>>> state store is closed on not -- I
>>>>>>>>>>>>>>>>>> can't query it anyway?
>>>>> Maybe
>>>>>>>>>>> I
>>>>>>>>>>>>>> miss
>>>>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Yes, both
>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>>>>> are fatal exception. But
>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`
>>>>>>>>>>>>>>>>> is fatal exception
>>> about
>>>>>>>>>>> state
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>> related. I think it would be
>>>>>>>>>>>>>>>>> helpful that if user need to
>>> distinguish
>>>>>>>>>>> these
>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>> different case to handle it.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'm not very sure, does that make
>>>>>>>>>>>>>>>>> sense?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM
>>>>>>>>>>>>>>>>> Vinoth Chandar <
>>>>>>>>>>> vinoth@apache.org>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> +1 on merging
>>>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`,
>>>>>>>>>>>>>>>>>> both exceptions are
>>> fatal
>>>>>>>>>>>>>> anyway. IMO
>>>>>>>>>>>>>>>>>> its best to have these exceptions
>>>>>>>>>>>>>>>>>> be about the state store
>>>>>>>>>>> (and not
>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>> state), to easier understanding.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Additionally, KIP-535 allows for
>>>>>>>>>>>>>>>>>> querying of state stores
>>> in
>>>>>>>>>>>>>> rebalancing
>>>>>>>>>>>>>>>>>> state. So do we need the
>>>>>>>>>>>>>>>>>> StreamsRebalancingException?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias
>>>>>>>>>>>>>>>>>> J. Sax" <
>>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> Sorry that I dropped the ball
>>>>>>>>>>>>>>>>>>> on this...
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP.
>>>>>>>>>>>>>>>>>>> Overall LGTM now. Feel free
>>> to
>>>>>>>>>>> start
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> VOTE
>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> What is still unclear to me is,
>>>>>>>>>>>>>>>>>>> what we gain by having
>>> both
>>>>>>>>>>>>>>>>>>> `StreamsNotRunningException`
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> `StateStoreNotAvailableException`.
>>>>>>>>>>>>>>>> Both
>>>>>>>>>>>>>>>>>>> exception are thrown when
>>>>>>>>>>>>>>>>>>> KafkaStreams is in state
>>>>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as
>>>>>>>>>>>>>>>>>>> a user what do I gain to
>>> know
>>>>>>>>>>> if
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> state store is closed on not --
>>>>>>>>>>>>>>>>>>> I can't query it anyway?
>>>>>>>>>>> Maybe I
>>>>>>>>>>>>>> miss
>>>>>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> Sorry for the late reply,
>>>>>>>>>>>>>>>>>>>> thanks for the review.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
Why is it only thrown if the state is REBALANCING? A
>>> store
>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>> migrated during a
>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
>>>>>>>>>>>>>>>>>>>>> Streams might
>>>>> resume
>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>> RUNNING state and afterward
>>>>>>>>>>>>>>>>>>>>> somebody tries to use an
>>> old
>>>>>>>>>>> store
>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>>>> Also, if state is
>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
>>>>>>>>>>>>>>>>>>>>> Hence, I think
>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>>>>> does only make sense
>>> during
>>>>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thank you point this, already
>>>>>>>>>>>>>>>>>>>> updated.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Why do we need to distinguish
>>>>>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
`KafkaStreamsNotRunningException` may be caused by
>>> various
>>>>>>>>>>>>>> reasons, I
>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>> it would be helpful that the
>>>>>>>>>>>>>>>>>>>> user can distinguish whether
>>>>>>>>>>>>>>>>>>>> it is caused by the state
>>>>> store
>>>>>>>>>>>>>> closed.
>>>>>>>>>>>>>>>>>>>> (Maybe I am wrong...)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Last, why do we distinguish
>>>>>>>>>>>>>>>>>>>> between `KafkaStreams`
>>> instance
>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it
>>>>>>>>>>>>>>>>>>>>> seems we should always
>>> refer to
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>>>>> because that is the level
>>>>>>>>>>>>>>>>>>>>> of granularity in which we
>>>>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>>>>> IQ atm.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Totally agree. Do you mean
>>>>>>>>>>>>>>>>>>>> the naming of state store
>>>>>>>>>>> exceptions?
>>>>>>>>>>>>>>>>>>>> I don't have special reason
>>>>>>>>>>>>>>>>>>>> to distinguish these two.
>>>>>>>>>>>>>>>>>>>> Your suggestion look more
>>>>>>>>>>>>>>>>>>>> reasonable for the exception
>>>>>>>>>>> naming.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Last, for
>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
>>>>>>>>>>>>>>>>>>>> I would add
>>> that a
>>>>>>>>>>> user
>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> rediscover the store and
>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
>>>>>>>>>>>>>>>>>>>>> the
>>> store
>>>>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>>>>> invalid and a new store
>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
>>>>>>>>>>>>>>>>>>>>> That
>>> is
>>>>> a
>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
>>>>>>>>>>>>>>>>>>>>> that allows for
>>>>>>>>>>> "blind"
>>>>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>>>>> that either resolve (if the
>>>>>>>>>>>>>>>>>>>>> store is still on the same
>>>>>>>>>>> instance
>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
>>>>>>>>>>>>>>>>>>>>> changes to
>>>>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> store was migrated away
>>>>>>>>>>>>>>>>>>>>> during rebalancing).
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The KIP already updated,
>>>>>>>>>>>>>>>>>>>> please take a look. :)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48
>>>>>>>>>>>>>>>>>>>> PM Matthias J. Sax <
>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Any update on this KIP?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM,
>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>> Sorry for the late reply.
>>>>>>>>>>>>>>>>>>>>>> The 2.4 deadline kept us
>>> quite
>>>>>>>>>>> busy.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
Why is it only thrown if the state is REBALANCING? A
>>>>> store
>>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>>>>> migrated during a
>>>>>>>>>>>>>>>>>>>>>> rebalance, and Kafka
>>>>>>>>>>>>>>>>>>>>>> Streams might
>>>>> resume
>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>> RUNNING state and
>>>>>>>>>>>>>>>>>>>>>> afterward somebody tries
>>>>>>>>>>>>>>>>>>>>>> to use an
>>> old
>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>>>>> Also, if state is
>>>>>>>>>>>>>>>>>>>>>> REBALANCING, should we
>>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`?
>>>>>>>>>>>>>>>>>>>>>> Hence, I think
>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>>>>>> does only make sense
>>> during
>>>>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Why do we need to
>>>>>>>>>>>>>>>>>>>>>> distinguish between
>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Last, why do we
>>>>>>>>>>>>>>>>>>>>>> distinguish between
>>>>>>>>>>>>>>>>>>>>>> `KafkaStreams`
>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me,
>>>>>>>>>>>>>>>>>>>>>> it seems we should
>>>>>>>>>>>>>>>>>>>>>> always
>>> refer
>>>>> to
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>>>>>> because that is the level
>>>>>>>>>>>>>>>>>>>>>> of granularity in which
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>>>>> IQ
>>>>>>>>>>>>>>>>>>>>> atm.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Last, for
>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`,
>>>>>>>>>>>>>>>>>>>>>> I would add
>>>>> that a
>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>> rediscover the store and
>>>>>>>>>>>>>>>>>>>>>> cannot blindly retry as
>>>>>>>>>>>>>>>>>>>>>> the
>>>>> store
>>>>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>>>>>> invalid and a new store
>>>>>>>>>>>>>>>>>>>>>> handle must be retrieved.
>>>>>>>>>>>>>>>>>>>>>> That
>>>>> is a
>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`
>>>>>>>>>>>>>>>>>>>>>> that allows for
>>>>>>>>>>> "blind"
>>>>>>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>>>>>> that either resolve (if
>>>>>>>>>>>>>>>>>>>>>> the store is still on the
>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>> instance
>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>> rebalancing finishes, or
>>>>>>>>>>>>>>>>>>>>>> changes to
>>>>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> store was migrated away
>>>>>>>>>>>>>>>>>>>>>> during rebalancing).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito
>>>>>>>>>>>>>>>>>>>>>> Jeng wrote:
>>>>>>>>>>>>>>>>>>>>>>> My bad. The short link
>>>>>>>>>>>>>>>>>>>>>>> `https://shorturl.at/CDNT9`
>>>
>>>>>>>>>>>>>>>>>>>>>>>
<https://shorturl.at/CDNT9>
>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
<https://shorturl.at/CDNT9> seems incorrect.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Please use the
>>>>>>>>>>>>>>>>>>>>>>> following instead:
>>>>>>>>>>> https://shorturl.at/bkKQU
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at
>>>>>>>>>>>>>>>>>>>>>>> 10:53 AM Vito Jeng <
>>>>>>>>>>>>>> vito@is-land.com.tw>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
Thank you for explanation. I agree with your
>>> opinion.
>>>>>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()`
>>>>>>>>>>>>>>>>>>>>>>>> would never throw
>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
For the case that corresponding thread crashes
>>> after we
>>>>>>>>>>>>>> handed out
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> store handle. We may
>>>>>>>>>>>>>>>>>>>>>>>> throw
>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
In `StreamThreadStateStoreProvider`, we would throw
>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>> when stream
>>> thread is
>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> running(
>>>>>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9)
>>>>>>>>>>>>>>>>>>>>>>>> or throw
>>>>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>> store is
>>>>>>>>>>>>>>>>>>>>>>>> closed(https://shorturl.at/hrvAN).
>>>>>>>>>>>>>>>>>>>>>>>> So I
>>> think
>>>>>>>>>>> we
>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> add a new type for
>>>>>>>>>>>>>>>>>>>>>>>> this case. Does that
>>>>>>>>>>>>>>>>>>>>>>>> make sense?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>>>>>>>> vs
>>>>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
I understand your point. I rename
>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
About check unknown state store names:
>>>>>>>>>>>>>>>>>>>>>>>> Thank you for the
>>>>>>>>>>>>>>>>>>>>>>>> hint. I add a new
>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>> `UnknownStateStoreException`
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> this case.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Also, we should
>>>>>>>>>>>>>>>>>>>>>>>>> still have fatal
>>>>>>>>>>>>>>>>>>>>>>>>> exception
>>>>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>>>>>> Not sure why you
>>>>>>>>>>> remove
>>>>>>>>>>>>>> it?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thank you point this,
>>>>>>>>>>>>>>>>>>>>>>>> already add it
>>>>>>>>>>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> The KIP already
>>>>>>>>>>>>>>>>>>>>>>>> updated, please take
>>>>>>>>>>>>>>>>>>>>>>>> a look.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> --- Vito
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>>
>
-----BEGIN PGP SIGNATURE-----

iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl45vCwACgkQO4miYXKq
/OhT4Q//elvNj2BRIMd7SqyQhpa6sSq4k2iG3wFlr/B2xkmkZPWUICt3SpCisIxG
RlP5ml7Mi8IcWv9jmMux9C1NpftboLudxIUcun/I4cP0T3S7ytXfzdO+V1WuD9xe
hfeqMwJTytusEv5VO3MyY+jYGqmPJrPJRViZ2Uwsj3Ojk0SZRb/m+b6ah6BGMh0E
nlUIux7cabg1OZ/ee0x21hcAB32lVIRfPLKZeHCTHuYmlW76X4pXhFxOm69Pumtj
ciVMr/pl7B+XiMk0C2Po04zQcP8+5/O7LYU4e8ha12NtuNuDTlQLGUb+S9Qrbxb2
xdFrlC81RROmIdIOTLiJeeBqhzd19llDgMhBf4spJMOeLn7359PX6r/9tDqonJrQ
wpSp4S54MDYfTPHUPqA6u/FsE0BX4EMU71ckC9rXVRRUnQ8A8cMk/6qWlRu8NvxQ
IiYuSo48UENlcTHxXC/rzIDp54gvtv2iml8QcOg6tS+hzhAI4yoY+1w4pnjPmRo0
4D1hnq5bd1SLrGSGcCxVbN0jtwgkcr50HBb1UkIDRndqapfKwZOMV65tIwTvxaaZ
r2QPYyPd6ZDgeMdy0r94z0SMwatqGCmJD9EQmSmLBulemrvYxPGUevQls+WgE4WE
zc3LtoyfdlXlkttTU0Q/dm2H9OU4s3QO+6PEdC+Qwi3aCKKhdUQ=
=4EMC
-----END PGP SIGNATURE-----


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Hi, folks,

KIP-562(KAFKA-9445) already merged three days ago.

I have updated KIP-216 to reflect the KIP-562.
The main change is to introduce a new exception
`InvalidStateStorePartitionException`, will be thrown when user requested
partition not available.

Please take a look and any feedback is welcome.
Thanks Matthias for the reminder.

---
Vito


On Thu, Jan 23, 2020 at 2:13 PM Vito Jeng <vi...@is-land.com.tw> wrote:

> Got it, thanks Matthias.
>
> ---
> Vito
>
>
> On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> Thanks Vito.
>>
>> I am also ok with either name. Just a personal slight preference, but
>> not a important.
>>
>>
>> -Matthias
>>
>> On 1/21/20 6:52 PM, Vito Jeng wrote:
>> > Thanks Matthias.
>> >
>> > The KIP is about InvalidStateStoreException.
>> > I pick `StateStoreNotAvailableException` because it may be more
>> intuitive
>> > than `StreamsNotRunningException`.
>> >
>> > No matter which one picked, it's good to me.
>> >
>> > ---
>> > Vito
>> >
>> >
>> > On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax <ma...@confluent.io>
>> > wrote:
>> >
>> >> Thanks for updating the KIP!
>> >>
>> >> One last comment/question: you kept `StateStoreNotAvailableException`
>> in
>> >> favor of `StreamsNotRunningException` (to merge both as suggested).
>> >>
>> >> I am wondering, if it might be better to keep
>> >> `StreamsNotRunningException` instead of
>> >> `StateStoreNotAvailableException`, because this exception is thrown if
>> >> Streams is in state PENDING_SHUTDOWN / NOT_RUNNING / ERROR ?
>> >>
>> >>
>> >>
>> >> -Matthias
>> >>
>> >> On 1/17/20 9:56 PM, John Roesler wrote:
>> >>> Thanks, Vito. I've just cast my vote.
>> >>> -John
>> >>>
>> >>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
>> >>>> Hi, folks,
>> >>>>
>> >>>> Just update the KIP, please take a look.
>> >>>>
>> >>>> Thanks!
>> >>>>
>> >>>> ---
>> >>>> Vito
>> >>>>
>> >>>>
>> >>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng <vi...@is-land.com.tw>
>> wrote:
>> >>>>
>> >>>>> Thanks Bill, John and Matthias. Glad you guys joined this
>> discussion.
>> >>>>> I got a lot out of the discussion.
>> >>>>>
>> >>>>> I would like to update KIP-216 base on John's suggestion to remove
>> the
>> >>>>> category.
>> >>>>>
>> >>>>>
>> >>>>> ---
>> >>>>> Vito
>> >>>>>
>> >>>>>
>> >>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
>> matthias@confluent.io
>> >>>
>> >>>>> wrote:
>> >>>>>
>> >>>>>>> Nevertheless, if we omit the categorization, it’s moot.
>> >>>>>>
>> >>>>>> Ack.
>> >>>>>>
>> >>>>>> I am fine to remove the middle tier. As John pointed out, it might
>> be
>> >>>>>> weird to have only one concrete exception type per category. We can
>> >> also
>> >>>>>> explain in detail how to handle each exception in their JavaDocs.
>> >>>>>>
>> >>>>>>
>> >>>>>> -Matthias
>> >>>>>>
>> >>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
>> >>>>>>> Vito,
>> >>>>>>>
>> >>>>>>> Thanks for the updates, the KIP LGTM.
>> >>>>>>>
>> >>>>>>> -Bill
>> >>>>>>>
>> >>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <
>> vvcephei@apache.org>
>> >>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> Hi Vito,
>> >>>>>>>>
>> >>>>>>>> Haha, your archive game is on point!
>> >>>>>>>>
>> >>>>>>>> What Matthias said in that email is essentially what I figured
>> was
>> >> the
>> >>>>>>>> rationale. It makes sense, but the point I was making is that
>> this
>> >>>>>> really
>> >>>>>>>> doesn’t seem like a good way to structure a production app. On
>> the
>> >>>>>> other
>> >>>>>>>> hand, considering the exception fatal has a good chance of
>> avoiding
>> >> a
>> >>>>>>>> frustrating debug session if you just forgot to call start.
>> >>>>>>>>
>> >>>>>>>> Nevertheless, if we omit the categorization, it’s moot.
>> >>>>>>>>
>> >>>>>>>> It would be easy to add a categorization layer later if we want
>> it,
>> >> but
>> >>>>>>>> not very easy to change it if we get it wrong.
>> >>>>>>>>
>> >>>>>>>> Thanks for your consideration!
>> >>>>>>>> -John
>> >>>>>>>>
>> >>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
>> >>>>>>>>> Hi John,
>> >>>>>>>>>
>> >>>>>>>>> About `StreamsNotStartedException is strange` --
>> >>>>>>>>> The original idea came from Matthias, two years ago. :)
>> >>>>>>>>> You can reference here:
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>
>> >>
>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
>> >>>>>>>>>
>> >>>>>>>>> About omitting the categorization --
>> >>>>>>>>> It looks reasonable. I'm fine with omitting the categorization
>> but
>> >> not
>> >>>>>>>> very
>> >>>>>>>>> sure it is a good choice.
>> >>>>>>>>> Does any other folks provide opinion?
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> Hi, folks,
>> >>>>>>>>>
>> >>>>>>>>> Just update the KIP-216, please take a look.
>> >>>>>>>>>
>> >>>>>>>>> ---
>> >>>>>>>>> Vito
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw>
>> >>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Hi, folks,
>> >>>>>>>>>>
>> >>>>>>>>>> Thank you suggestion, really appreciate it. :)
>> >>>>>>>>>> I understand your concern. I'll merge
>> StreamsNotRunningException
>> >> and
>> >>>>>>>>>> StateStoreNotAvailableException.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> ---
>> >>>>>>>>>> Vito
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <
>> vvcephei@apache.org
>> >>>
>> >>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>> Hey Vito,
>> >>>>>>>>>>>
>> >>>>>>>>>>> Yes, thanks for the KIP. Sorry the discussion has been so
>> long.
>> >>>>>>>>>>> Hopefully, we can close it out soon.
>> >>>>>>>>>>>
>> >>>>>>>>>>> I agree we can drop StreamsNotRunningException in favor of
>> >>>>>>>>>>> just StateStoreNotAvailableException.
>> >>>>>>>>>>>
>> >>>>>>>>>>> Unfortunately, I have some higher-level concerns. The value
>> >>>>>>>>>>> of these exceptions is that they tell you how to handle the
>> >>>>>>>>>>> various situations that can arise while querying a distributed
>> >>>>>>>>>>> data store.
>> >>>>>>>>>>>
>> >>>>>>>>>>> Ideally, as a caller, I should be able to just catch
>> "retriable"
>> >> or
>> >>>>>>>>>>> "fatal" and handle them appropriately. Otherwise, there's no
>> >>>>>>>>>>> point in having categories, and we should just have all the
>> >>>>>>>>>>> exceptions extend InvalidStateStoreException.
>> >>>>>>>>>>>
>> >>>>>>>>>>> Presently, it's not possible to tell from just the
>> >>>>>>>>>>> "retriable"/"fatal" distinction what to do. You  can tell
>> >>>>>>>>>>> from the descriptions of the various exceptions. E.g.:
>> >>>>>>>>>>>
>> >>>>>>>>>>> Retriable:
>> >>>>>>>>>>>  * StreamsRebalancingException: the exact same call
>> >>>>>>>>>>>     should just be retried until the rebalance is complete
>> >>>>>>>>>>>  * StateStoreMigratedException: the store handle is
>> >>>>>>>>>>>     now invalid, so you need to re-discover the instance
>> >>>>>>>>>>>     and get a new handle on that instance. In other words,
>> >>>>>>>>>>>     the query itself may be valid, but the particular method
>> >>>>>>>>>>>     invocation on this particular instance has encountered
>> >>>>>>>>>>>     a fatal exception.
>> >>>>>>>>>>>
>> >>>>>>>>>>> Fatal:
>> >>>>>>>>>>>  * UnknownStateStoreException: this is truly fatal. No amount
>> >>>>>>>>>>>     of retrying or re-discovering is going to get you a handle
>> >> on a
>> >>>>>>>>>>>     store that doesn't exist in the cluster.
>> >>>>>>>>>>>  * StateStoreNotAvailableException: this is actually
>> recoverable,
>> >>>>>>>>>>>     since the store might exist in the cluster, but isn't
>> >> available
>> >>>>>> on
>> >>>>>>>>>>>     this particular instance (which is shut down or whatever).
>> >>>>>>>>>>>
>> >>>>>>>>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100%
>> fine
>> >>>>>>>>>>> with omitting the categorization and just having 5 subclasses
>> >>>>>>>>>>> of InvalidStateStoreException. Each of them would tell you
>> >>>>>>>>>>> how to handle them, and it's not too many to really
>> >>>>>>>>>>> understand and handle each one.
>> >>>>>>>>>>>
>> >>>>>>>>>>> If you really want to have a middle tier, I'd recommend:
>> >>>>>>>>>>> * RetryableStateStoreException: the exact same call
>> >>>>>>>>>>>     should be repeated.
>> >>>>>>>>>>> * RecoverableStateStoreException: the store handle
>> >>>>>>>>>>>     should be discarded and the caller should re-discover
>> >>>>>>>>>>>     the location of the store and repeat the query on the
>> >>>>>>>>>>>     correct instance.
>> >>>>>>>>>>> * FatalStateStoreException: the query/request is totally
>> >>>>>>>>>>>     invalid and will never succeed.
>> >>>>>>>>>>>
>> >>>>>>>>>>> However, attempting to categorize the proposed exceptions
>> >>>>>>>>>>> reveals even problems with this categorization:
>> >>>>>>>>>>> Retriable:
>> >>>>>>>>>>> * StreamsRebalancingException
>> >>>>>>>>>>> Recoverable:
>> >>>>>>>>>>> * StateStoreMigratedException
>> >>>>>>>>>>> * StreamsNotRunningException
>> >>>>>>>>>>> Fatal:
>> >>>>>>>>>>> * UnknownStateStoreException
>> >>>>>>>>>>>
>> >>>>>>>>>>> But StreamsNotStartedException is strange... It means that
>> >>>>>>>>>>> one code path got a handle on a specific KafkaStreams object
>> >>>>>>>>>>> instance and sent it a query before another code path
>> >>>>>>>>>>> invoked the start() method on the exact same object instance.
>> >>>>>>>>>>> It seems like the most likely scenario is that whoever wrote
>> >>>>>>>>>>> the program just forgot to call start() before querying, in
>> >>>>>>>>>>> which case, retrying isn't going to help, and a fatal
>> exception
>> >>>>>>>>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
>> >>>>>>>>>>> experience" problem, and making it fatal would be more
>> >>>>>>>>>>> helpful. Even in a production context, there's no reason not
>> >>>>>>>>>>> to sequence your application startup such that you don't
>> >>>>>>>>>>> accept queries until after Streams is started. Thus, I guess
>> >>>>>>>>>>> I'd categorize it under "fatal".
>> >>>>>>>>>>>
>> >>>>>>>>>>> Regardless of whether you make it fatal or retriable, you'd
>> >>>>>>>>>>> still have a whole category with only one exception in it,
>> >>>>>>>>>>> and the other two categories only have two exceptions.
>> >>>>>>>>>>> Plus, as you pointed out in the KIP, you can't get all
>> >>>>>>>>>>> exceptions in all cases anyway:
>> >>>>>>>>>>> * store() can only throw NotStarted, NotRunning,
>> >>>>>>>>>>>     and Unknown
>> >>>>>>>>>>> * actual store queries can only throw Rebalancing,
>> >>>>>>>>>>>     Migrated, and NotRunning
>> >>>>>>>>>>>
>> >>>>>>>>>>> Thus, in practice also, there are exactly three categories
>> >>>>>>>>>>> and also exactly three exception types. It doesn't seem
>> >>>>>>>>>>> like there's a great advantage to the categories here. To
>> >>>>>>>>>>> avoid the categorization problem and also to clarify what
>> >>>>>>>>>>> exceptions can actually be thrown in different circumstances,
>> >>>>>>>>>>> it seems like we should just:
>> >>>>>>>>>>> * get rid of the middle tier and make all the exceptions
>> >>>>>>>>>>>     extend InvalidStateStoreException
>> >>>>>>>>>>> * drop StateStoreNotAvailableException in favor of
>> >>>>>>>>>>>     StreamsNotRunningException
>> >>>>>>>>>>> * clearly document on all public methods which exceptions
>> >>>>>>>>>>>     need to be handled
>> >>>>>>>>>>>
>> >>>>>>>>>>> How do you feel about this?
>> >>>>>>>>>>> Thanks,
>> >>>>>>>>>>> -John
>> >>>>>>>>>>>
>> >>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
>> >>>>>>>>>>>> Thanks for KIP Vito.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Overall the KIP LGTM, but I'd have to agree with others on
>> >> merging
>> >>>>>>>> the
>> >>>>>>>>>>>> `StreamsNotRunningException` and
>> >> `StateStoreNotAvailableException`
>> >>>>>>>>>>> classes.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Since in both cases, the thread state is in
>> `PENDING_SHUTDOWN ||
>> >>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could
>> distinguish
>> >>>>>>>> when to
>> >>>>>>>>>>>> use the different
>> >>>>>>>>>>>> exceptions.  Maybe a good middle ground would be to have a
>> >> detailed
>> >>>>>>>>>>>> exception message.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> The KIP freeze is close, so I think if we can agree on this,
>> we
>> >> can
>> >>>>>>>>>>> wrap up
>> >>>>>>>>>>>> the voting soon.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Thanks,
>> >>>>>>>>>>>> Bill
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
>> >>>>>>>> matthias@confluent.io>
>> >>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>> Vito,
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> It's still unclear to me what the advantage is, to have both
>> >>>>>>>>>>>>> `StreamsNotRunningException` and
>> >>>>>>>> `StateStoreNotAvailableException`?
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> For both cased, the state is `PENDING_SHUTDOWN /
>> NOT_RUNNING /
>> >>>>>>>> ERROR`
>> >>>>>>>>>>>>> and thus, for a user point of view, why does it matter if
>> the
>> >>>>>>>> store is
>> >>>>>>>>>>>>> closed on not? I don't understand why/how this information
>> >> would
>> >>>>>>>> be
>> >>>>>>>>>>>>> useful? Do you have a concrete example in mind how a user
>> would
>> >>>>>>>> react
>> >>>>>>>>>>>>> differently to both exceptions?
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it
>> seems
>> >>>>>>>> best
>> >>>>>>>>>>> to
>> >>>>>>>>>>>>> actually do this on a per-query basis, ie, have an overload
>> >>>>>>>>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that
>> allow
>> >> to
>> >>>>>>>>>>>>> _disable_ the exception and opt-in to query a active store
>> >> during
>> >>>>>>>>>>>>> recovery. However, as KIP-535 actually introduces this
>> change
>> >> in
>> >>>>>>>>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
>> >>>>>>>> should be
>> >>>>>>>>>>>>> updated. I'll follow up on the other KIP thread to raise
>> this
>> >>>>>>>> point.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> -Matthias
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
>> >>>>>>>>>>>>>> Hi, Matthias & Vinoth,
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> Thanks for the feedback.
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> What is still unclear to me is, what we gain by having
>> both
>> >>>>>>>>>>>>>>> `StreamsNotRunningException` and
>> >>>>>>>>>>> `StateStoreNotAvailableException`. Both
>> >>>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
>> >>>>>>>>>>> PENDING_SHUTDOWN /
>> >>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to
>> know
>> >>>>>>>> if the
>> >>>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
>> >> Maybe
>> >>>>>>>> I
>> >>>>>>>>>>> miss
>> >>>>>>>>>>>>>>> something thought?
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> Yes, both `StreamsNotRunningException` and
>> >>>>>>>>>>>>>> `StateStoreNotAvailableException` are fatal exception.
>> >>>>>>>>>>>>>> But `StateStoreNotAvailableException` is fatal exception
>> about
>> >>>>>>>> state
>> >>>>>>>>>>>>> store
>> >>>>>>>>>>>>>> related.
>> >>>>>>>>>>>>>> I think it would be helpful that if user need to
>> distinguish
>> >>>>>>>> these
>> >>>>>>>>>>> two
>> >>>>>>>>>>>>>> different case to handle it.
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> I'm not very sure, does that make sense?
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> ---
>> >>>>>>>>>>>>>> Vito
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
>> >>>>>>>> vinoth@apache.org>
>> >>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> +1 on merging `StreamsNotRunningException` and
>> >>>>>>>>>>>>>>> `StateStoreNotAvailableException`, both exceptions are
>> fatal
>> >>>>>>>>>>> anyway. IMO
>> >>>>>>>>>>>>>>> its best to have these exceptions be about the state store
>> >>>>>>>> (and not
>> >>>>>>>>>>>>> streams
>> >>>>>>>>>>>>>>> state), to easier understanding.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Additionally, KIP-535 allows for querying of state stores
>> in
>> >>>>>>>>>>> rebalancing
>> >>>>>>>>>>>>>>> state. So do we need the StreamsRebalancingException?
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
>> >>>>>>>> matthias@confluent.io>
>> >>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>> Sorry that I dropped the ball on this...
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free
>> to
>> >>>>>>>> start
>> >>>>>>>>>>> a
>> >>>>>>>>>>>>> VOTE
>> >>>>>>>>>>>>>>>> thread.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> What is still unclear to me is, what we gain by having
>> both
>> >>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>> >>>>>>>>>>> `StateStoreNotAvailableException`.
>> >>>>>>>>>>>>> Both
>> >>>>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
>> >>>>>>>>>>> PENDING_SHUTDOWN /
>> >>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to
>> know
>> >>>>>>>> if
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
>> >>>>>>>> Maybe I
>> >>>>>>>>>>> miss
>> >>>>>>>>>>>>>>>> something thought?
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> -Matthias
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>> >>>>>>>>>>>>>>>>> Sorry for the late reply, thanks for the review.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A
>> store
>> >>>>>>>>>>> might be
>> >>>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
>> >> resume
>> >>>>>>>>>>> back to
>> >>>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an
>> old
>> >>>>>>>> store
>> >>>>>>>>>>>>> handle.
>> >>>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>> >>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>> >>>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense
>> during
>> >>>>>>>>>>> `RUNNING`
>> >>>>>>>>>>>>>>> state.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Thank you point this, already updated.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Why do we need to distinguish between
>> >>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>> >>>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by
>> various
>> >>>>>>>>>>> reasons, I
>> >>>>>>>>>>>>>>> think
>> >>>>>>>>>>>>>>>>> it would be helpful that the
>> >>>>>>>>>>>>>>>>> user can distinguish whether it is caused by the state
>> >> store
>> >>>>>>>>>>> closed.
>> >>>>>>>>>>>>>>>>> (Maybe I am wrong...)
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
>> instance
>> >>>>>>>> and
>> >>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always
>> refer to
>> >>>>>>>> the
>> >>>>>>>>>>>>>>> instance,
>> >>>>>>>>>>>>>>>>>> because that is the level of granularity in which we
>> >>>>>>>>>>> enable/disable
>> >>>>>>>>>>>>>>> IQ atm.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Totally agree. Do you mean the naming of state store
>> >>>>>>>> exceptions?
>> >>>>>>>>>>>>>>>>> I don't have special reason to distinguish these two.
>> >>>>>>>>>>>>>>>>> Your suggestion look more reasonable for the exception
>> >>>>>>>> naming.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add
>> that a
>> >>>>>>>> user
>> >>>>>>>>>>> need
>> >>>>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the
>> store
>> >>>>>>>>>>> handle is
>> >>>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That
>> is
>> >> a
>> >>>>>>>>>>>>> difference
>> >>>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>> >>>>>>>> "blind"
>> >>>>>>>>>>> retries
>> >>>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
>> >>>>>>>> instance
>> >>>>>>>>>>> after
>> >>>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
>> >>>>>>>>>>> `StateStoreMigratedException` if
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> The KIP already updated, please take a look. :)
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
>> >>>>>>>>>>>>> matthias@confluent.io
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> Any update on this KIP?
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>> >>>>>>>>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us
>> quite
>> >>>>>>>> busy.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A
>> >> store
>> >>>>>>>>>>> might be
>> >>>>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
>> >> resume
>> >>>>>>>>>>> back to
>> >>>>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an
>> old
>> >>>>>>>> store
>> >>>>>>>>>>>>>>> handle.
>> >>>>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>> >>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>> >>>>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense
>> during
>> >>>>>>>>>>> `RUNNING`
>> >>>>>>>>>>>>>>>>>> state.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Why do we need to distinguish between
>> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>> >>>>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
>> >>>>>>>> instance and
>> >>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always
>> refer
>> >> to
>> >>>>>>>> the
>> >>>>>>>>>>>>>>> instance,
>> >>>>>>>>>>>>>>>>>>> because that is the level of granularity in which we
>> >>>>>>>>>>> enable/disable
>> >>>>>>>>>>>>>>> IQ
>> >>>>>>>>>>>>>>>>>> atm.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add
>> >> that a
>> >>>>>>>>>>> user
>> >>>>>>>>>>>>>>> need to
>> >>>>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the
>> >> store
>> >>>>>>>>>>> handle is
>> >>>>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That
>> >> is a
>> >>>>>>>>>>>>>>> difference
>> >>>>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>> >>>>>>>> "blind"
>> >>>>>>>>>>>>> retries
>> >>>>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
>> >>>>>>>> instance
>> >>>>>>>>>>>>> after
>> >>>>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
>> >>>>>>>>>>> `StateStoreMigratedException` if
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> -Matthias
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>> >>>>>>>>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9>
>> >> <https://shorturl.at/CDNT9>
>> >>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Please use the following instead:
>> >>>>>>>> https://shorturl.at/bkKQU
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> ---
>> >>>>>>>>>>>>>>>>>>>> Vito
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
>> >>>>>>>>>>> vito@is-land.com.tw>
>> >>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Thank you for explanation. I agree with your
>> opinion.
>> >>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>> >>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> For the case that corresponding thread crashes
>> after we
>> >>>>>>>>>>> handed out
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>>>>> store handle. We may throw
>> >>>>>>>> `KafkaStreamsNotRunningException`
>> >>>>>>>>>>> or
>> >>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
>> >>>>>>>>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>> >>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream
>> thread is
>> >>>>>>>> not
>> >>>>>>>>>>>>>>> running(
>> >>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
>> >>>>>>>>>>> `StateStoreMigratedException`
>> >>>>>>>>>>>>>>> when
>> >>>>>>>>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I
>> think
>> >>>>>>>> we
>> >>>>>>>>>>> do not
>> >>>>>>>>>>>>>>> need
>> >>>>>>>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
>> >>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> I understand your point. I rename
>> >>>>>>>>>>>>>>> `StreamThreadNotRunningException` to
>> >>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> About check unknown state store names:
>> >>>>>>>>>>>>>>>>>>>>> Thank you for the hint. I add a new type
>> >>>>>>>>>>>>>>> `UnknownStateStoreException`
>> >>>>>>>>>>>>>>>>>> for
>> >>>>>>>>>>>>>>>>>>>>> this case.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> Also, we should still have fatal exception
>> >>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
>> >>>>>>>> remove
>> >>>>>>>>>>> it?
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Thank you point this, already add it again.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> The KIP already updated, please take a look.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> ---
>> >>>>>>>>>>>>>>>>>>>>> Vito
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>
>> >>
>> >>
>> >
>>
>>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Got it, thanks Matthias.

---
Vito


On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Thanks Vito.
>
> I am also ok with either name. Just a personal slight preference, but
> not a important.
>
>
> -Matthias
>
> On 1/21/20 6:52 PM, Vito Jeng wrote:
> > Thanks Matthias.
> >
> > The KIP is about InvalidStateStoreException.
> > I pick `StateStoreNotAvailableException` because it may be more intuitive
> > than `StreamsNotRunningException`.
> >
> > No matter which one picked, it's good to me.
> >
> > ---
> > Vito
> >
> >
> > On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> Thanks for updating the KIP!
> >>
> >> One last comment/question: you kept `StateStoreNotAvailableException` in
> >> favor of `StreamsNotRunningException` (to merge both as suggested).
> >>
> >> I am wondering, if it might be better to keep
> >> `StreamsNotRunningException` instead of
> >> `StateStoreNotAvailableException`, because this exception is thrown if
> >> Streams is in state PENDING_SHUTDOWN / NOT_RUNNING / ERROR ?
> >>
> >>
> >>
> >> -Matthias
> >>
> >> On 1/17/20 9:56 PM, John Roesler wrote:
> >>> Thanks, Vito. I've just cast my vote.
> >>> -John
> >>>
> >>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
> >>>> Hi, folks,
> >>>>
> >>>> Just update the KIP, please take a look.
> >>>>
> >>>> Thanks!
> >>>>
> >>>> ---
> >>>> Vito
> >>>>
> >>>>
> >>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng <vi...@is-land.com.tw>
> wrote:
> >>>>
> >>>>> Thanks Bill, John and Matthias. Glad you guys joined this discussion.
> >>>>> I got a lot out of the discussion.
> >>>>>
> >>>>> I would like to update KIP-216 base on John's suggestion to remove
> the
> >>>>> category.
> >>>>>
> >>>>>
> >>>>> ---
> >>>>> Vito
> >>>>>
> >>>>>
> >>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <
> matthias@confluent.io
> >>>
> >>>>> wrote:
> >>>>>
> >>>>>>> Nevertheless, if we omit the categorization, it’s moot.
> >>>>>>
> >>>>>> Ack.
> >>>>>>
> >>>>>> I am fine to remove the middle tier. As John pointed out, it might
> be
> >>>>>> weird to have only one concrete exception type per category. We can
> >> also
> >>>>>> explain in detail how to handle each exception in their JavaDocs.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
> >>>>>>> Vito,
> >>>>>>>
> >>>>>>> Thanks for the updates, the KIP LGTM.
> >>>>>>>
> >>>>>>> -Bill
> >>>>>>>
> >>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vvcephei@apache.org
> >
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Vito,
> >>>>>>>>
> >>>>>>>> Haha, your archive game is on point!
> >>>>>>>>
> >>>>>>>> What Matthias said in that email is essentially what I figured was
> >> the
> >>>>>>>> rationale. It makes sense, but the point I was making is that this
> >>>>>> really
> >>>>>>>> doesn’t seem like a good way to structure a production app. On the
> >>>>>> other
> >>>>>>>> hand, considering the exception fatal has a good chance of
> avoiding
> >> a
> >>>>>>>> frustrating debug session if you just forgot to call start.
> >>>>>>>>
> >>>>>>>> Nevertheless, if we omit the categorization, it’s moot.
> >>>>>>>>
> >>>>>>>> It would be easy to add a categorization layer later if we want
> it,
> >> but
> >>>>>>>> not very easy to change it if we get it wrong.
> >>>>>>>>
> >>>>>>>> Thanks for your consideration!
> >>>>>>>> -John
> >>>>>>>>
> >>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
> >>>>>>>>> Hi John,
> >>>>>>>>>
> >>>>>>>>> About `StreamsNotStartedException is strange` --
> >>>>>>>>> The original idea came from Matthias, two years ago. :)
> >>>>>>>>> You can reference here:
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>
> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> >>>>>>>>>
> >>>>>>>>> About omitting the categorization --
> >>>>>>>>> It looks reasonable. I'm fine with omitting the categorization
> but
> >> not
> >>>>>>>> very
> >>>>>>>>> sure it is a good choice.
> >>>>>>>>> Does any other folks provide opinion?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Hi, folks,
> >>>>>>>>>
> >>>>>>>>> Just update the KIP-216, please take a look.
> >>>>>>>>>
> >>>>>>>>> ---
> >>>>>>>>> Vito
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw>
> >>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Hi, folks,
> >>>>>>>>>>
> >>>>>>>>>> Thank you suggestion, really appreciate it. :)
> >>>>>>>>>> I understand your concern. I'll merge StreamsNotRunningException
> >> and
> >>>>>>>>>> StateStoreNotAvailableException.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> ---
> >>>>>>>>>> Vito
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <
> vvcephei@apache.org
> >>>
> >>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hey Vito,
> >>>>>>>>>>>
> >>>>>>>>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
> >>>>>>>>>>> Hopefully, we can close it out soon.
> >>>>>>>>>>>
> >>>>>>>>>>> I agree we can drop StreamsNotRunningException in favor of
> >>>>>>>>>>> just StateStoreNotAvailableException.
> >>>>>>>>>>>
> >>>>>>>>>>> Unfortunately, I have some higher-level concerns. The value
> >>>>>>>>>>> of these exceptions is that they tell you how to handle the
> >>>>>>>>>>> various situations that can arise while querying a distributed
> >>>>>>>>>>> data store.
> >>>>>>>>>>>
> >>>>>>>>>>> Ideally, as a caller, I should be able to just catch
> "retriable"
> >> or
> >>>>>>>>>>> "fatal" and handle them appropriately. Otherwise, there's no
> >>>>>>>>>>> point in having categories, and we should just have all the
> >>>>>>>>>>> exceptions extend InvalidStateStoreException.
> >>>>>>>>>>>
> >>>>>>>>>>> Presently, it's not possible to tell from just the
> >>>>>>>>>>> "retriable"/"fatal" distinction what to do. You  can tell
> >>>>>>>>>>> from the descriptions of the various exceptions. E.g.:
> >>>>>>>>>>>
> >>>>>>>>>>> Retriable:
> >>>>>>>>>>>  * StreamsRebalancingException: the exact same call
> >>>>>>>>>>>     should just be retried until the rebalance is complete
> >>>>>>>>>>>  * StateStoreMigratedException: the store handle is
> >>>>>>>>>>>     now invalid, so you need to re-discover the instance
> >>>>>>>>>>>     and get a new handle on that instance. In other words,
> >>>>>>>>>>>     the query itself may be valid, but the particular method
> >>>>>>>>>>>     invocation on this particular instance has encountered
> >>>>>>>>>>>     a fatal exception.
> >>>>>>>>>>>
> >>>>>>>>>>> Fatal:
> >>>>>>>>>>>  * UnknownStateStoreException: this is truly fatal. No amount
> >>>>>>>>>>>     of retrying or re-discovering is going to get you a handle
> >> on a
> >>>>>>>>>>>     store that doesn't exist in the cluster.
> >>>>>>>>>>>  * StateStoreNotAvailableException: this is actually
> recoverable,
> >>>>>>>>>>>     since the store might exist in the cluster, but isn't
> >> available
> >>>>>> on
> >>>>>>>>>>>     this particular instance (which is shut down or whatever).
> >>>>>>>>>>>
> >>>>>>>>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> >>>>>>>>>>> with omitting the categorization and just having 5 subclasses
> >>>>>>>>>>> of InvalidStateStoreException. Each of them would tell you
> >>>>>>>>>>> how to handle them, and it's not too many to really
> >>>>>>>>>>> understand and handle each one.
> >>>>>>>>>>>
> >>>>>>>>>>> If you really want to have a middle tier, I'd recommend:
> >>>>>>>>>>> * RetryableStateStoreException: the exact same call
> >>>>>>>>>>>     should be repeated.
> >>>>>>>>>>> * RecoverableStateStoreException: the store handle
> >>>>>>>>>>>     should be discarded and the caller should re-discover
> >>>>>>>>>>>     the location of the store and repeat the query on the
> >>>>>>>>>>>     correct instance.
> >>>>>>>>>>> * FatalStateStoreException: the query/request is totally
> >>>>>>>>>>>     invalid and will never succeed.
> >>>>>>>>>>>
> >>>>>>>>>>> However, attempting to categorize the proposed exceptions
> >>>>>>>>>>> reveals even problems with this categorization:
> >>>>>>>>>>> Retriable:
> >>>>>>>>>>> * StreamsRebalancingException
> >>>>>>>>>>> Recoverable:
> >>>>>>>>>>> * StateStoreMigratedException
> >>>>>>>>>>> * StreamsNotRunningException
> >>>>>>>>>>> Fatal:
> >>>>>>>>>>> * UnknownStateStoreException
> >>>>>>>>>>>
> >>>>>>>>>>> But StreamsNotStartedException is strange... It means that
> >>>>>>>>>>> one code path got a handle on a specific KafkaStreams object
> >>>>>>>>>>> instance and sent it a query before another code path
> >>>>>>>>>>> invoked the start() method on the exact same object instance.
> >>>>>>>>>>> It seems like the most likely scenario is that whoever wrote
> >>>>>>>>>>> the program just forgot to call start() before querying, in
> >>>>>>>>>>> which case, retrying isn't going to help, and a fatal exception
> >>>>>>>>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
> >>>>>>>>>>> experience" problem, and making it fatal would be more
> >>>>>>>>>>> helpful. Even in a production context, there's no reason not
> >>>>>>>>>>> to sequence your application startup such that you don't
> >>>>>>>>>>> accept queries until after Streams is started. Thus, I guess
> >>>>>>>>>>> I'd categorize it under "fatal".
> >>>>>>>>>>>
> >>>>>>>>>>> Regardless of whether you make it fatal or retriable, you'd
> >>>>>>>>>>> still have a whole category with only one exception in it,
> >>>>>>>>>>> and the other two categories only have two exceptions.
> >>>>>>>>>>> Plus, as you pointed out in the KIP, you can't get all
> >>>>>>>>>>> exceptions in all cases anyway:
> >>>>>>>>>>> * store() can only throw NotStarted, NotRunning,
> >>>>>>>>>>>     and Unknown
> >>>>>>>>>>> * actual store queries can only throw Rebalancing,
> >>>>>>>>>>>     Migrated, and NotRunning
> >>>>>>>>>>>
> >>>>>>>>>>> Thus, in practice also, there are exactly three categories
> >>>>>>>>>>> and also exactly three exception types. It doesn't seem
> >>>>>>>>>>> like there's a great advantage to the categories here. To
> >>>>>>>>>>> avoid the categorization problem and also to clarify what
> >>>>>>>>>>> exceptions can actually be thrown in different circumstances,
> >>>>>>>>>>> it seems like we should just:
> >>>>>>>>>>> * get rid of the middle tier and make all the exceptions
> >>>>>>>>>>>     extend InvalidStateStoreException
> >>>>>>>>>>> * drop StateStoreNotAvailableException in favor of
> >>>>>>>>>>>     StreamsNotRunningException
> >>>>>>>>>>> * clearly document on all public methods which exceptions
> >>>>>>>>>>>     need to be handled
> >>>>>>>>>>>
> >>>>>>>>>>> How do you feel about this?
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> -John
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> >>>>>>>>>>>> Thanks for KIP Vito.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Overall the KIP LGTM, but I'd have to agree with others on
> >> merging
> >>>>>>>> the
> >>>>>>>>>>>> `StreamsNotRunningException` and
> >> `StateStoreNotAvailableException`
> >>>>>>>>>>> classes.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN
> ||
> >>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could
> distinguish
> >>>>>>>> when to
> >>>>>>>>>>>> use the different
> >>>>>>>>>>>> exceptions.  Maybe a good middle ground would be to have a
> >> detailed
> >>>>>>>>>>>> exception message.
> >>>>>>>>>>>>
> >>>>>>>>>>>> The KIP freeze is close, so I think if we can agree on this,
> we
> >> can
> >>>>>>>>>>> wrap up
> >>>>>>>>>>>> the voting soon.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>> Bill
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
> >>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Vito,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> It's still unclear to me what the advantage is, to have both
> >>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING
> /
> >>>>>>>> ERROR`
> >>>>>>>>>>>>> and thus, for a user point of view, why does it matter if the
> >>>>>>>> store is
> >>>>>>>>>>>>> closed on not? I don't understand why/how this information
> >> would
> >>>>>>>> be
> >>>>>>>>>>>>> useful? Do you have a concrete example in mind how a user
> would
> >>>>>>>> react
> >>>>>>>>>>>>> differently to both exceptions?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it
> seems
> >>>>>>>> best
> >>>>>>>>>>> to
> >>>>>>>>>>>>> actually do this on a per-query basis, ie, have an overload
> >>>>>>>>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that
> allow
> >> to
> >>>>>>>>>>>>> _disable_ the exception and opt-in to query a active store
> >> during
> >>>>>>>>>>>>> recovery. However, as KIP-535 actually introduces this change
> >> in
> >>>>>>>>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
> >>>>>>>> should be
> >>>>>>>>>>>>> updated. I'll follow up on the other KIP thread to raise this
> >>>>>>>> point.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
> >>>>>>>>>>>>>> Hi, Matthias & Vinoth,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> What is still unclear to me is, what we gain by having both
> >>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>> `StateStoreNotAvailableException`. Both
> >>>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
> >>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to
> know
> >>>>>>>> if the
> >>>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
> >> Maybe
> >>>>>>>> I
> >>>>>>>>>>> miss
> >>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Yes, both `StreamsNotRunningException` and
> >>>>>>>>>>>>>> `StateStoreNotAvailableException` are fatal exception.
> >>>>>>>>>>>>>> But `StateStoreNotAvailableException` is fatal exception
> about
> >>>>>>>> state
> >>>>>>>>>>>>> store
> >>>>>>>>>>>>>> related.
> >>>>>>>>>>>>>> I think it would be helpful that if user need to distinguish
> >>>>>>>> these
> >>>>>>>>>>> two
> >>>>>>>>>>>>>> different case to handle it.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I'm not very sure, does that make sense?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
> >>>>>>>> vinoth@apache.org>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> +1 on merging `StreamsNotRunningException` and
> >>>>>>>>>>>>>>> `StateStoreNotAvailableException`, both exceptions are
> fatal
> >>>>>>>>>>> anyway. IMO
> >>>>>>>>>>>>>>> its best to have these exceptions be about the state store
> >>>>>>>> (and not
> >>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>> state), to easier understanding.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Additionally, KIP-535 allows for querying of state stores
> in
> >>>>>>>>>>> rebalancing
> >>>>>>>>>>>>>>> state. So do we need the StreamsRebalancingException?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
> >>>>>>>> matthias@confluent.io>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>> Sorry that I dropped the ball on this...
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free
> to
> >>>>>>>> start
> >>>>>>>>>>> a
> >>>>>>>>>>>>> VOTE
> >>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> What is still unclear to me is, what we gain by having
> both
> >>>>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>>>> `StateStoreNotAvailableException`.
> >>>>>>>>>>>>> Both
> >>>>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
> >>>>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to
> know
> >>>>>>>> if
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
> >>>>>>>> Maybe I
> >>>>>>>>>>> miss
> >>>>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> >>>>>>>>>>>>>>>>> Sorry for the late reply, thanks for the review.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A
> store
> >>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
> >> resume
> >>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> >>>>>>>> store
> >>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense
> during
> >>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thank you point this, already updated.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Why do we need to distinguish between
> >>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by
> various
> >>>>>>>>>>> reasons, I
> >>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>> it would be helpful that the
> >>>>>>>>>>>>>>>>> user can distinguish whether it is caused by the state
> >> store
> >>>>>>>>>>> closed.
> >>>>>>>>>>>>>>>>> (Maybe I am wrong...)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
> instance
> >>>>>>>> and
> >>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer
> to
> >>>>>>>> the
> >>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>> because that is the level of granularity in which we
> >>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>> IQ atm.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Totally agree. Do you mean the naming of state store
> >>>>>>>> exceptions?
> >>>>>>>>>>>>>>>>> I don't have special reason to distinguish these two.
> >>>>>>>>>>>>>>>>> Your suggestion look more reasonable for the exception
> >>>>>>>> naming.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add
> that a
> >>>>>>>> user
> >>>>>>>>>>> need
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the
> store
> >>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That
> is
> >> a
> >>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >>>>>>>> "blind"
> >>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
> >>>>>>>> instance
> >>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
> >>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> The KIP already updated, please take a look. :)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> >>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Any update on this KIP?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us
> quite
> >>>>>>>> busy.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A
> >> store
> >>>>>>>>>>> might be
> >>>>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
> >> resume
> >>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an
> old
> >>>>>>>> store
> >>>>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense
> during
> >>>>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Why do we need to distinguish between
> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
> >>>>>>>> instance and
> >>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer
> >> to
> >>>>>>>> the
> >>>>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>>>> because that is the level of granularity in which we
> >>>>>>>>>>> enable/disable
> >>>>>>>>>>>>>>> IQ
> >>>>>>>>>>>>>>>>>> atm.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add
> >> that a
> >>>>>>>>>>> user
> >>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the
> >> store
> >>>>>>>>>>> handle is
> >>>>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That
> >> is a
> >>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >>>>>>>> "blind"
> >>>>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
> >>>>>>>> instance
> >>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
> >>>>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >>>>>>>>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >> <https://shorturl.at/CDNT9>
> >>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Please use the following instead:
> >>>>>>>> https://shorturl.at/bkKQU
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
> >>>>>>>>>>> vito@is-land.com.tw>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
> >>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> For the case that corresponding thread crashes after
> we
> >>>>>>>>>>> handed out
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> store handle. We may throw
> >>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
> >>>>>>>>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread
> is
> >>>>>>>> not
> >>>>>>>>>>>>>>> running(
> >>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
> >>>>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I
> think
> >>>>>>>> we
> >>>>>>>>>>> do not
> >>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
> >>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I understand your point. I rename
> >>>>>>>>>>>>>>> `StreamThreadNotRunningException` to
> >>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> About check unknown state store names:
> >>>>>>>>>>>>>>>>>>>>> Thank you for the hint. I add a new type
> >>>>>>>>>>>>>>> `UnknownStateStoreException`
> >>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>> this case.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Also, we should still have fatal exception
> >>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
> >>>>>>>> remove
> >>>>>>>>>>> it?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thank you point this, already add it again.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> The KIP already updated, please take a look.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>
> >>
> >
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks Vito.

I am also ok with either name. Just a personal slight preference, but
not a important.


-Matthias

On 1/21/20 6:52 PM, Vito Jeng wrote:
> Thanks Matthias.
> 
> The KIP is about InvalidStateStoreException.
> I pick `StateStoreNotAvailableException` because it may be more intuitive
> than `StreamsNotRunningException`.
> 
> No matter which one picked, it's good to me.
> 
> ---
> Vito
> 
> 
> On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> Thanks for updating the KIP!
>>
>> One last comment/question: you kept `StateStoreNotAvailableException` in
>> favor of `StreamsNotRunningException` (to merge both as suggested).
>>
>> I am wondering, if it might be better to keep
>> `StreamsNotRunningException` instead of
>> `StateStoreNotAvailableException`, because this exception is thrown if
>> Streams is in state PENDING_SHUTDOWN / NOT_RUNNING / ERROR ?
>>
>>
>>
>> -Matthias
>>
>> On 1/17/20 9:56 PM, John Roesler wrote:
>>> Thanks, Vito. I've just cast my vote.
>>> -John
>>>
>>> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
>>>> Hi, folks,
>>>>
>>>> Just update the KIP, please take a look.
>>>>
>>>> Thanks!
>>>>
>>>> ---
>>>> Vito
>>>>
>>>>
>>>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng <vi...@is-land.com.tw> wrote:
>>>>
>>>>> Thanks Bill, John and Matthias. Glad you guys joined this discussion.
>>>>> I got a lot out of the discussion.
>>>>>
>>>>> I would like to update KIP-216 base on John's suggestion to remove the
>>>>> category.
>>>>>
>>>>>
>>>>> ---
>>>>> Vito
>>>>>
>>>>>
>>>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <matthias@confluent.io
>>>
>>>>> wrote:
>>>>>
>>>>>>> Nevertheless, if we omit the categorization, it’s moot.
>>>>>>
>>>>>> Ack.
>>>>>>
>>>>>> I am fine to remove the middle tier. As John pointed out, it might be
>>>>>> weird to have only one concrete exception type per category. We can
>> also
>>>>>> explain in detail how to handle each exception in their JavaDocs.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
>>>>>>> Vito,
>>>>>>>
>>>>>>> Thanks for the updates, the KIP LGTM.
>>>>>>>
>>>>>>> -Bill
>>>>>>>
>>>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Vito,
>>>>>>>>
>>>>>>>> Haha, your archive game is on point!
>>>>>>>>
>>>>>>>> What Matthias said in that email is essentially what I figured was
>> the
>>>>>>>> rationale. It makes sense, but the point I was making is that this
>>>>>> really
>>>>>>>> doesn’t seem like a good way to structure a production app. On the
>>>>>> other
>>>>>>>> hand, considering the exception fatal has a good chance of avoiding
>> a
>>>>>>>> frustrating debug session if you just forgot to call start.
>>>>>>>>
>>>>>>>> Nevertheless, if we omit the categorization, it’s moot.
>>>>>>>>
>>>>>>>> It would be easy to add a categorization layer later if we want it,
>> but
>>>>>>>> not very easy to change it if we get it wrong.
>>>>>>>>
>>>>>>>> Thanks for your consideration!
>>>>>>>> -John
>>>>>>>>
>>>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
>>>>>>>>> Hi John,
>>>>>>>>>
>>>>>>>>> About `StreamsNotStartedException is strange` --
>>>>>>>>> The original idea came from Matthias, two years ago. :)
>>>>>>>>> You can reference here:
>>>>>>>>>
>>>>>>>>
>>>>>>
>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
>>>>>>>>>
>>>>>>>>> About omitting the categorization --
>>>>>>>>> It looks reasonable. I'm fine with omitting the categorization but
>> not
>>>>>>>> very
>>>>>>>>> sure it is a good choice.
>>>>>>>>> Does any other folks provide opinion?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Hi, folks,
>>>>>>>>>
>>>>>>>>> Just update the KIP-216, please take a look.
>>>>>>>>>
>>>>>>>>> ---
>>>>>>>>> Vito
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw>
>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Hi, folks,
>>>>>>>>>>
>>>>>>>>>> Thank you suggestion, really appreciate it. :)
>>>>>>>>>> I understand your concern. I'll merge StreamsNotRunningException
>> and
>>>>>>>>>> StateStoreNotAvailableException.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> ---
>>>>>>>>>> Vito
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vvcephei@apache.org
>>>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hey Vito,
>>>>>>>>>>>
>>>>>>>>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
>>>>>>>>>>> Hopefully, we can close it out soon.
>>>>>>>>>>>
>>>>>>>>>>> I agree we can drop StreamsNotRunningException in favor of
>>>>>>>>>>> just StateStoreNotAvailableException.
>>>>>>>>>>>
>>>>>>>>>>> Unfortunately, I have some higher-level concerns. The value
>>>>>>>>>>> of these exceptions is that they tell you how to handle the
>>>>>>>>>>> various situations that can arise while querying a distributed
>>>>>>>>>>> data store.
>>>>>>>>>>>
>>>>>>>>>>> Ideally, as a caller, I should be able to just catch "retriable"
>> or
>>>>>>>>>>> "fatal" and handle them appropriately. Otherwise, there's no
>>>>>>>>>>> point in having categories, and we should just have all the
>>>>>>>>>>> exceptions extend InvalidStateStoreException.
>>>>>>>>>>>
>>>>>>>>>>> Presently, it's not possible to tell from just the
>>>>>>>>>>> "retriable"/"fatal" distinction what to do. You  can tell
>>>>>>>>>>> from the descriptions of the various exceptions. E.g.:
>>>>>>>>>>>
>>>>>>>>>>> Retriable:
>>>>>>>>>>>  * StreamsRebalancingException: the exact same call
>>>>>>>>>>>     should just be retried until the rebalance is complete
>>>>>>>>>>>  * StateStoreMigratedException: the store handle is
>>>>>>>>>>>     now invalid, so you need to re-discover the instance
>>>>>>>>>>>     and get a new handle on that instance. In other words,
>>>>>>>>>>>     the query itself may be valid, but the particular method
>>>>>>>>>>>     invocation on this particular instance has encountered
>>>>>>>>>>>     a fatal exception.
>>>>>>>>>>>
>>>>>>>>>>> Fatal:
>>>>>>>>>>>  * UnknownStateStoreException: this is truly fatal. No amount
>>>>>>>>>>>     of retrying or re-discovering is going to get you a handle
>> on a
>>>>>>>>>>>     store that doesn't exist in the cluster.
>>>>>>>>>>>  * StateStoreNotAvailableException: this is actually recoverable,
>>>>>>>>>>>     since the store might exist in the cluster, but isn't
>> available
>>>>>> on
>>>>>>>>>>>     this particular instance (which is shut down or whatever).
>>>>>>>>>>>
>>>>>>>>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
>>>>>>>>>>> with omitting the categorization and just having 5 subclasses
>>>>>>>>>>> of InvalidStateStoreException. Each of them would tell you
>>>>>>>>>>> how to handle them, and it's not too many to really
>>>>>>>>>>> understand and handle each one.
>>>>>>>>>>>
>>>>>>>>>>> If you really want to have a middle tier, I'd recommend:
>>>>>>>>>>> * RetryableStateStoreException: the exact same call
>>>>>>>>>>>     should be repeated.
>>>>>>>>>>> * RecoverableStateStoreException: the store handle
>>>>>>>>>>>     should be discarded and the caller should re-discover
>>>>>>>>>>>     the location of the store and repeat the query on the
>>>>>>>>>>>     correct instance.
>>>>>>>>>>> * FatalStateStoreException: the query/request is totally
>>>>>>>>>>>     invalid and will never succeed.
>>>>>>>>>>>
>>>>>>>>>>> However, attempting to categorize the proposed exceptions
>>>>>>>>>>> reveals even problems with this categorization:
>>>>>>>>>>> Retriable:
>>>>>>>>>>> * StreamsRebalancingException
>>>>>>>>>>> Recoverable:
>>>>>>>>>>> * StateStoreMigratedException
>>>>>>>>>>> * StreamsNotRunningException
>>>>>>>>>>> Fatal:
>>>>>>>>>>> * UnknownStateStoreException
>>>>>>>>>>>
>>>>>>>>>>> But StreamsNotStartedException is strange... It means that
>>>>>>>>>>> one code path got a handle on a specific KafkaStreams object
>>>>>>>>>>> instance and sent it a query before another code path
>>>>>>>>>>> invoked the start() method on the exact same object instance.
>>>>>>>>>>> It seems like the most likely scenario is that whoever wrote
>>>>>>>>>>> the program just forgot to call start() before querying, in
>>>>>>>>>>> which case, retrying isn't going to help, and a fatal exception
>>>>>>>>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
>>>>>>>>>>> experience" problem, and making it fatal would be more
>>>>>>>>>>> helpful. Even in a production context, there's no reason not
>>>>>>>>>>> to sequence your application startup such that you don't
>>>>>>>>>>> accept queries until after Streams is started. Thus, I guess
>>>>>>>>>>> I'd categorize it under "fatal".
>>>>>>>>>>>
>>>>>>>>>>> Regardless of whether you make it fatal or retriable, you'd
>>>>>>>>>>> still have a whole category with only one exception in it,
>>>>>>>>>>> and the other two categories only have two exceptions.
>>>>>>>>>>> Plus, as you pointed out in the KIP, you can't get all
>>>>>>>>>>> exceptions in all cases anyway:
>>>>>>>>>>> * store() can only throw NotStarted, NotRunning,
>>>>>>>>>>>     and Unknown
>>>>>>>>>>> * actual store queries can only throw Rebalancing,
>>>>>>>>>>>     Migrated, and NotRunning
>>>>>>>>>>>
>>>>>>>>>>> Thus, in practice also, there are exactly three categories
>>>>>>>>>>> and also exactly three exception types. It doesn't seem
>>>>>>>>>>> like there's a great advantage to the categories here. To
>>>>>>>>>>> avoid the categorization problem and also to clarify what
>>>>>>>>>>> exceptions can actually be thrown in different circumstances,
>>>>>>>>>>> it seems like we should just:
>>>>>>>>>>> * get rid of the middle tier and make all the exceptions
>>>>>>>>>>>     extend InvalidStateStoreException
>>>>>>>>>>> * drop StateStoreNotAvailableException in favor of
>>>>>>>>>>>     StreamsNotRunningException
>>>>>>>>>>> * clearly document on all public methods which exceptions
>>>>>>>>>>>     need to be handled
>>>>>>>>>>>
>>>>>>>>>>> How do you feel about this?
>>>>>>>>>>> Thanks,
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
>>>>>>>>>>>> Thanks for KIP Vito.
>>>>>>>>>>>>
>>>>>>>>>>>> Overall the KIP LGTM, but I'd have to agree with others on
>> merging
>>>>>>>> the
>>>>>>>>>>>> `StreamsNotRunningException` and
>> `StateStoreNotAvailableException`
>>>>>>>>>>> classes.
>>>>>>>>>>>>
>>>>>>>>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
>>>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
>>>>>>>> when to
>>>>>>>>>>>> use the different
>>>>>>>>>>>> exceptions.  Maybe a good middle ground would be to have a
>> detailed
>>>>>>>>>>>> exception message.
>>>>>>>>>>>>
>>>>>>>>>>>> The KIP freeze is close, so I think if we can agree on this, we
>> can
>>>>>>>>>>> wrap up
>>>>>>>>>>>> the voting soon.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Bill
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Vito,
>>>>>>>>>>>>>
>>>>>>>>>>>>> It's still unclear to me what the advantage is, to have both
>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>>>
>>>>>>>>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
>>>>>>>> ERROR`
>>>>>>>>>>>>> and thus, for a user point of view, why does it matter if the
>>>>>>>> store is
>>>>>>>>>>>>> closed on not? I don't understand why/how this information
>> would
>>>>>>>> be
>>>>>>>>>>>>> useful? Do you have a concrete example in mind how a user would
>>>>>>>> react
>>>>>>>>>>>>> differently to both exceptions?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
>>>>>>>> best
>>>>>>>>>>> to
>>>>>>>>>>>>> actually do this on a per-query basis, ie, have an overload
>>>>>>>>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow
>> to
>>>>>>>>>>>>> _disable_ the exception and opt-in to query a active store
>> during
>>>>>>>>>>>>> recovery. However, as KIP-535 actually introduces this change
>> in
>>>>>>>>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
>>>>>>>> should be
>>>>>>>>>>>>> updated. I'll follow up on the other KIP thread to raise this
>>>>>>>> point.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
>>>>>>>>>>>>>> Hi, Matthias & Vinoth,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> What is still unclear to me is, what we gain by having both
>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>> `StateStoreNotAvailableException`. Both
>>>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>>>>>>>> if the
>>>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
>> Maybe
>>>>>>>> I
>>>>>>>>>>> miss
>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, both `StreamsNotRunningException` and
>>>>>>>>>>>>>> `StateStoreNotAvailableException` are fatal exception.
>>>>>>>>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
>>>>>>>> state
>>>>>>>>>>>>> store
>>>>>>>>>>>>>> related.
>>>>>>>>>>>>>> I think it would be helpful that if user need to distinguish
>>>>>>>> these
>>>>>>>>>>> two
>>>>>>>>>>>>>> different case to handle it.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm not very sure, does that make sense?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ---
>>>>>>>>>>>>>> Vito
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
>>>>>>>> vinoth@apache.org>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> +1 on merging `StreamsNotRunningException` and
>>>>>>>>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
>>>>>>>>>>> anyway. IMO
>>>>>>>>>>>>>>> its best to have these exceptions be about the state store
>>>>>>>> (and not
>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>> state), to easier understanding.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
>>>>>>>>>>> rebalancing
>>>>>>>>>>>>>>> state. So do we need the StreamsRebalancingException?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> Sorry that I dropped the ball on this...
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
>>>>>>>> start
>>>>>>>>>>> a
>>>>>>>>>>>>> VOTE
>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What is still unclear to me is, what we gain by having both
>>>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>>>> `StateStoreNotAvailableException`.
>>>>>>>>>>>>> Both
>>>>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
>>>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>>>>>>>> if
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
>>>>>>>> Maybe I
>>>>>>>>>>> miss
>>>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>>>>>>>>>>>>>>>>> Sorry for the late reply, thanks for the review.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
>> resume
>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>>>>>>>> store
>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thank you point this, already updated.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Why do we need to distinguish between
>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
>>>>>>>>>>> reasons, I
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>> it would be helpful that the
>>>>>>>>>>>>>>>>> user can distinguish whether it is caused by the state
>> store
>>>>>>>>>>> closed.
>>>>>>>>>>>>>>>>> (Maybe I am wrong...)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
>>>>>>>> and
>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
>>>>>>>> the
>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>> because that is the level of granularity in which we
>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>> IQ atm.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Totally agree. Do you mean the naming of state store
>>>>>>>> exceptions?
>>>>>>>>>>>>>>>>> I don't have special reason to distinguish these two.
>>>>>>>>>>>>>>>>> Your suggestion look more reasonable for the exception
>>>>>>>> naming.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
>>>>>>>> user
>>>>>>>>>>> need
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is
>> a
>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>>>>>>>> "blind"
>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
>>>>>>>> instance
>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Nice, it's great! Thank you.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The KIP already updated, please take a look. :)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Any update on this KIP?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
>>>>>>>> busy.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A
>> store
>>>>>>>>>>> might be
>>>>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
>> resume
>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>>>>>>>> store
>>>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>>>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>>>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Why do we need to distinguish between
>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer
>> to
>>>>>>>> the
>>>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>>>> because that is the level of granularity in which we
>>>>>>>>>>> enable/disable
>>>>>>>>>>>>>>> IQ
>>>>>>>>>>>>>>>>>> atm.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add
>> that a
>>>>>>>>>>> user
>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the
>> store
>>>>>>>>>>> handle is
>>>>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That
>> is a
>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>>>>>>>> "blind"
>>>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
>>>>>>>> instance
>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
>>>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>>>>>>>>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9>
>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Please use the following instead:
>>>>>>>> https://shorturl.at/bkKQU
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>> Vito
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
>>>>>>>>>>> vito@is-land.com.tw>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
>>>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
>>>>>>>>>>> handed out
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> store handle. We may throw
>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
>>>>>>>>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
>>>>>>>> not
>>>>>>>>>>>>>>> running(
>>>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
>>>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
>>>>>>>> we
>>>>>>>>>>> do not
>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
>>>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I understand your point. I rename
>>>>>>>>>>>>>>> `StreamThreadNotRunningException` to
>>>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> About check unknown state store names:
>>>>>>>>>>>>>>>>>>>>> Thank you for the hint. I add a new type
>>>>>>>>>>>>>>> `UnknownStateStoreException`
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> this case.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Also, we should still have fatal exception
>>>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
>>>>>>>> remove
>>>>>>>>>>> it?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thank you point this, already add it again.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The KIP already updated, please take a look.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>> Vito
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>
>>
>>
> 


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Thanks Matthias.

The KIP is about InvalidStateStoreException.
I pick `StateStoreNotAvailableException` because it may be more intuitive
than `StreamsNotRunningException`.

No matter which one picked, it's good to me.

---
Vito


On Wed, Jan 22, 2020 at 7:44 AM Matthias J. Sax <ma...@confluent.io>
wrote:

> Thanks for updating the KIP!
>
> One last comment/question: you kept `StateStoreNotAvailableException` in
> favor of `StreamsNotRunningException` (to merge both as suggested).
>
> I am wondering, if it might be better to keep
> `StreamsNotRunningException` instead of
> `StateStoreNotAvailableException`, because this exception is thrown if
> Streams is in state PENDING_SHUTDOWN / NOT_RUNNING / ERROR ?
>
>
>
> -Matthias
>
> On 1/17/20 9:56 PM, John Roesler wrote:
> > Thanks, Vito. I've just cast my vote.
> > -John
> >
> > On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
> >> Hi, folks,
> >>
> >> Just update the KIP, please take a look.
> >>
> >> Thanks!
> >>
> >> ---
> >> Vito
> >>
> >>
> >> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> >>
> >>> Thanks Bill, John and Matthias. Glad you guys joined this discussion.
> >>> I got a lot out of the discussion.
> >>>
> >>> I would like to update KIP-216 base on John's suggestion to remove the
> >>> category.
> >>>
> >>>
> >>> ---
> >>> Vito
> >>>
> >>>
> >>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <matthias@confluent.io
> >
> >>> wrote:
> >>>
> >>>>> Nevertheless, if we omit the categorization, it’s moot.
> >>>>
> >>>> Ack.
> >>>>
> >>>> I am fine to remove the middle tier. As John pointed out, it might be
> >>>> weird to have only one concrete exception type per category. We can
> also
> >>>> explain in detail how to handle each exception in their JavaDocs.
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
> >>>>> Vito,
> >>>>>
> >>>>> Thanks for the updates, the KIP LGTM.
> >>>>>
> >>>>> -Bill
> >>>>>
> >>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org>
> >>>> wrote:
> >>>>>
> >>>>>> Hi Vito,
> >>>>>>
> >>>>>> Haha, your archive game is on point!
> >>>>>>
> >>>>>> What Matthias said in that email is essentially what I figured was
> the
> >>>>>> rationale. It makes sense, but the point I was making is that this
> >>>> really
> >>>>>> doesn’t seem like a good way to structure a production app. On the
> >>>> other
> >>>>>> hand, considering the exception fatal has a good chance of avoiding
> a
> >>>>>> frustrating debug session if you just forgot to call start.
> >>>>>>
> >>>>>> Nevertheless, if we omit the categorization, it’s moot.
> >>>>>>
> >>>>>> It would be easy to add a categorization layer later if we want it,
> but
> >>>>>> not very easy to change it if we get it wrong.
> >>>>>>
> >>>>>> Thanks for your consideration!
> >>>>>> -John
> >>>>>>
> >>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
> >>>>>>> Hi John,
> >>>>>>>
> >>>>>>> About `StreamsNotStartedException is strange` --
> >>>>>>> The original idea came from Matthias, two years ago. :)
> >>>>>>> You can reference here:
> >>>>>>>
> >>>>>>
> >>>>
> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> >>>>>>>
> >>>>>>> About omitting the categorization --
> >>>>>>> It looks reasonable. I'm fine with omitting the categorization but
> not
> >>>>>> very
> >>>>>>> sure it is a good choice.
> >>>>>>> Does any other folks provide opinion?
> >>>>>>>
> >>>>>>>
> >>>>>>> Hi, folks,
> >>>>>>>
> >>>>>>> Just update the KIP-216, please take a look.
> >>>>>>>
> >>>>>>> ---
> >>>>>>> Vito
> >>>>>>>
> >>>>>>>
> >>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw>
> >>>> wrote:
> >>>>>>>
> >>>>>>>>
> >>>>>>>> Hi, folks,
> >>>>>>>>
> >>>>>>>> Thank you suggestion, really appreciate it. :)
> >>>>>>>> I understand your concern. I'll merge StreamsNotRunningException
> and
> >>>>>>>> StateStoreNotAvailableException.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> ---
> >>>>>>>> Vito
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vvcephei@apache.org
> >
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hey Vito,
> >>>>>>>>>
> >>>>>>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
> >>>>>>>>> Hopefully, we can close it out soon.
> >>>>>>>>>
> >>>>>>>>> I agree we can drop StreamsNotRunningException in favor of
> >>>>>>>>> just StateStoreNotAvailableException.
> >>>>>>>>>
> >>>>>>>>> Unfortunately, I have some higher-level concerns. The value
> >>>>>>>>> of these exceptions is that they tell you how to handle the
> >>>>>>>>> various situations that can arise while querying a distributed
> >>>>>>>>> data store.
> >>>>>>>>>
> >>>>>>>>> Ideally, as a caller, I should be able to just catch "retriable"
> or
> >>>>>>>>> "fatal" and handle them appropriately. Otherwise, there's no
> >>>>>>>>> point in having categories, and we should just have all the
> >>>>>>>>> exceptions extend InvalidStateStoreException.
> >>>>>>>>>
> >>>>>>>>> Presently, it's not possible to tell from just the
> >>>>>>>>> "retriable"/"fatal" distinction what to do. You  can tell
> >>>>>>>>> from the descriptions of the various exceptions. E.g.:
> >>>>>>>>>
> >>>>>>>>> Retriable:
> >>>>>>>>>  * StreamsRebalancingException: the exact same call
> >>>>>>>>>     should just be retried until the rebalance is complete
> >>>>>>>>>  * StateStoreMigratedException: the store handle is
> >>>>>>>>>     now invalid, so you need to re-discover the instance
> >>>>>>>>>     and get a new handle on that instance. In other words,
> >>>>>>>>>     the query itself may be valid, but the particular method
> >>>>>>>>>     invocation on this particular instance has encountered
> >>>>>>>>>     a fatal exception.
> >>>>>>>>>
> >>>>>>>>> Fatal:
> >>>>>>>>>  * UnknownStateStoreException: this is truly fatal. No amount
> >>>>>>>>>     of retrying or re-discovering is going to get you a handle
> on a
> >>>>>>>>>     store that doesn't exist in the cluster.
> >>>>>>>>>  * StateStoreNotAvailableException: this is actually recoverable,
> >>>>>>>>>     since the store might exist in the cluster, but isn't
> available
> >>>> on
> >>>>>>>>>     this particular instance (which is shut down or whatever).
> >>>>>>>>>
> >>>>>>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> >>>>>>>>> with omitting the categorization and just having 5 subclasses
> >>>>>>>>> of InvalidStateStoreException. Each of them would tell you
> >>>>>>>>> how to handle them, and it's not too many to really
> >>>>>>>>> understand and handle each one.
> >>>>>>>>>
> >>>>>>>>> If you really want to have a middle tier, I'd recommend:
> >>>>>>>>> * RetryableStateStoreException: the exact same call
> >>>>>>>>>     should be repeated.
> >>>>>>>>> * RecoverableStateStoreException: the store handle
> >>>>>>>>>     should be discarded and the caller should re-discover
> >>>>>>>>>     the location of the store and repeat the query on the
> >>>>>>>>>     correct instance.
> >>>>>>>>> * FatalStateStoreException: the query/request is totally
> >>>>>>>>>     invalid and will never succeed.
> >>>>>>>>>
> >>>>>>>>> However, attempting to categorize the proposed exceptions
> >>>>>>>>> reveals even problems with this categorization:
> >>>>>>>>> Retriable:
> >>>>>>>>> * StreamsRebalancingException
> >>>>>>>>> Recoverable:
> >>>>>>>>> * StateStoreMigratedException
> >>>>>>>>> * StreamsNotRunningException
> >>>>>>>>> Fatal:
> >>>>>>>>> * UnknownStateStoreException
> >>>>>>>>>
> >>>>>>>>> But StreamsNotStartedException is strange... It means that
> >>>>>>>>> one code path got a handle on a specific KafkaStreams object
> >>>>>>>>> instance and sent it a query before another code path
> >>>>>>>>> invoked the start() method on the exact same object instance.
> >>>>>>>>> It seems like the most likely scenario is that whoever wrote
> >>>>>>>>> the program just forgot to call start() before querying, in
> >>>>>>>>> which case, retrying isn't going to help, and a fatal exception
> >>>>>>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
> >>>>>>>>> experience" problem, and making it fatal would be more
> >>>>>>>>> helpful. Even in a production context, there's no reason not
> >>>>>>>>> to sequence your application startup such that you don't
> >>>>>>>>> accept queries until after Streams is started. Thus, I guess
> >>>>>>>>> I'd categorize it under "fatal".
> >>>>>>>>>
> >>>>>>>>> Regardless of whether you make it fatal or retriable, you'd
> >>>>>>>>> still have a whole category with only one exception in it,
> >>>>>>>>> and the other two categories only have two exceptions.
> >>>>>>>>> Plus, as you pointed out in the KIP, you can't get all
> >>>>>>>>> exceptions in all cases anyway:
> >>>>>>>>> * store() can only throw NotStarted, NotRunning,
> >>>>>>>>>     and Unknown
> >>>>>>>>> * actual store queries can only throw Rebalancing,
> >>>>>>>>>     Migrated, and NotRunning
> >>>>>>>>>
> >>>>>>>>> Thus, in practice also, there are exactly three categories
> >>>>>>>>> and also exactly three exception types. It doesn't seem
> >>>>>>>>> like there's a great advantage to the categories here. To
> >>>>>>>>> avoid the categorization problem and also to clarify what
> >>>>>>>>> exceptions can actually be thrown in different circumstances,
> >>>>>>>>> it seems like we should just:
> >>>>>>>>> * get rid of the middle tier and make all the exceptions
> >>>>>>>>>     extend InvalidStateStoreException
> >>>>>>>>> * drop StateStoreNotAvailableException in favor of
> >>>>>>>>>     StreamsNotRunningException
> >>>>>>>>> * clearly document on all public methods which exceptions
> >>>>>>>>>     need to be handled
> >>>>>>>>>
> >>>>>>>>> How do you feel about this?
> >>>>>>>>> Thanks,
> >>>>>>>>> -John
> >>>>>>>>>
> >>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> >>>>>>>>>> Thanks for KIP Vito.
> >>>>>>>>>>
> >>>>>>>>>> Overall the KIP LGTM, but I'd have to agree with others on
> merging
> >>>>>> the
> >>>>>>>>>> `StreamsNotRunningException` and
> `StateStoreNotAvailableException`
> >>>>>>>>> classes.
> >>>>>>>>>>
> >>>>>>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> >>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
> >>>>>> when to
> >>>>>>>>>> use the different
> >>>>>>>>>> exceptions.  Maybe a good middle ground would be to have a
> detailed
> >>>>>>>>>> exception message.
> >>>>>>>>>>
> >>>>>>>>>> The KIP freeze is close, so I think if we can agree on this, we
> can
> >>>>>>>>> wrap up
> >>>>>>>>>> the voting soon.
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>> Bill
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
> >>>>>> matthias@confluent.io>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Vito,
> >>>>>>>>>>>
> >>>>>>>>>>> It's still unclear to me what the advantage is, to have both
> >>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>> `StateStoreNotAvailableException`?
> >>>>>>>>>>>
> >>>>>>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
> >>>>>> ERROR`
> >>>>>>>>>>> and thus, for a user point of view, why does it matter if the
> >>>>>> store is
> >>>>>>>>>>> closed on not? I don't understand why/how this information
> would
> >>>>>> be
> >>>>>>>>>>> useful? Do you have a concrete example in mind how a user would
> >>>>>> react
> >>>>>>>>>>> differently to both exceptions?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
> >>>>>> best
> >>>>>>>>> to
> >>>>>>>>>>> actually do this on a per-query basis, ie, have an overload
> >>>>>>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow
> to
> >>>>>>>>>>> _disable_ the exception and opt-in to query a active store
> during
> >>>>>>>>>>> recovery. However, as KIP-535 actually introduces this change
> in
> >>>>>>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
> >>>>>> should be
> >>>>>>>>>>> updated. I'll follow up on the other KIP thread to raise this
> >>>>>> point.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> -Matthias
> >>>>>>>>>>>
> >>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
> >>>>>>>>>>>> Hi, Matthias & Vinoth,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>
> >>>>>>>>>>>>> What is still unclear to me is, what we gain by having both
> >>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>> `StateStoreNotAvailableException`. Both
> >>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
> >>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> >>>>>> if the
> >>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
> Maybe
> >>>>>> I
> >>>>>>>>> miss
> >>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Yes, both `StreamsNotRunningException` and
> >>>>>>>>>>>> `StateStoreNotAvailableException` are fatal exception.
> >>>>>>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
> >>>>>> state
> >>>>>>>>>>> store
> >>>>>>>>>>>> related.
> >>>>>>>>>>>> I think it would be helpful that if user need to distinguish
> >>>>>> these
> >>>>>>>>> two
> >>>>>>>>>>>> different case to handle it.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I'm not very sure, does that make sense?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> ---
> >>>>>>>>>>>> Vito
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
> >>>>>> vinoth@apache.org>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> +1 on merging `StreamsNotRunningException` and
> >>>>>>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
> >>>>>>>>> anyway. IMO
> >>>>>>>>>>>>> its best to have these exceptions be about the state store
> >>>>>> (and not
> >>>>>>>>>>> streams
> >>>>>>>>>>>>> state), to easier understanding.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
> >>>>>>>>> rebalancing
> >>>>>>>>>>>>> state. So do we need the StreamsRebalancingException?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
> >>>>>> matthias@confluent.io>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>> Sorry that I dropped the ball on this...
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
> >>>>>> start
> >>>>>>>>> a
> >>>>>>>>>>> VOTE
> >>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> What is still unclear to me is, what we gain by having both
> >>>>>>>>>>>>>> `StreamsNotRunningException` and
> >>>>>>>>> `StateStoreNotAvailableException`.
> >>>>>>>>>>> Both
> >>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
> >>>>>>>>> PENDING_SHUTDOWN /
> >>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> >>>>>> if
> >>>>>>>>> the
> >>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
> >>>>>> Maybe I
> >>>>>>>>> miss
> >>>>>>>>>>>>>> something thought?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> >>>>>>>>>>>>>>> Sorry for the late reply, thanks for the review.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
> >>>>>>>>> might be
> >>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
> resume
> >>>>>>>>> back to
> >>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> >>>>>> store
> >>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> >>>>>>>>> `RUNNING`
> >>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thank you point this, already updated.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Why do we need to distinguish between
> >>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
> >>>>>>>>> reasons, I
> >>>>>>>>>>>>> think
> >>>>>>>>>>>>>>> it would be helpful that the
> >>>>>>>>>>>>>>> user can distinguish whether it is caused by the state
> store
> >>>>>>>>> closed.
> >>>>>>>>>>>>>>> (Maybe I am wrong...)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
> >>>>>> and
> >>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
> >>>>>> the
> >>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>> because that is the level of granularity in which we
> >>>>>>>>> enable/disable
> >>>>>>>>>>>>> IQ atm.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Totally agree. Do you mean the naming of state store
> >>>>>> exceptions?
> >>>>>>>>>>>>>>> I don't have special reason to distinguish these two.
> >>>>>>>>>>>>>>> Your suggestion look more reasonable for the exception
> >>>>>> naming.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
> >>>>>> user
> >>>>>>>>> need
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
> >>>>>>>>> handle is
> >>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is
> a
> >>>>>>>>>>> difference
> >>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >>>>>> "blind"
> >>>>>>>>> retries
> >>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
> >>>>>> instance
> >>>>>>>>> after
> >>>>>>>>>>>>>>>> rebalancing finishes, or changes to
> >>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> store was migrated away during rebalancing).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Nice, it's great! Thank you.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The KIP already updated, please take a look. :)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> >>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Any update on this KIP?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
> >>>>>> busy.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A
> store
> >>>>>>>>> might be
> >>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might
> resume
> >>>>>>>>> back to
> >>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> >>>>>> store
> >>>>>>>>>>>>> handle.
> >>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> >>>>>>>>> `RUNNING`
> >>>>>>>>>>>>>>>> state.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Why do we need to distinguish between
> >>>>>>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
> >>>>>> instance and
> >>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer
> to
> >>>>>> the
> >>>>>>>>>>>>> instance,
> >>>>>>>>>>>>>>>>> because that is the level of granularity in which we
> >>>>>>>>> enable/disable
> >>>>>>>>>>>>> IQ
> >>>>>>>>>>>>>>>> atm.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add
> that a
> >>>>>>>>> user
> >>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the
> store
> >>>>>>>>> handle is
> >>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That
> is a
> >>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >>>>>> "blind"
> >>>>>>>>>>> retries
> >>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
> >>>>>> instance
> >>>>>>>>>>> after
> >>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
> >>>>>>>>> `StateStoreMigratedException` if
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >>>>>>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >>>> <https://shorturl.at/CDNT9>
> >>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Please use the following instead:
> >>>>>> https://shorturl.at/bkKQU
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
> >>>>>>>>> vito@is-land.com.tw>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks, Matthias!
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
> >>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
> >>>>>>>>> handed out
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> store handle. We may throw
> >>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
> >>>>>>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
> >>>>>> not
> >>>>>>>>>>>>> running(
> >>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
> >>>>>>>>> `StateStoreMigratedException`
> >>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
> >>>>>> we
> >>>>>>>>> do not
> >>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
> >>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I understand your point. I rename
> >>>>>>>>>>>>> `StreamThreadNotRunningException` to
> >>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> About check unknown state store names:
> >>>>>>>>>>>>>>>>>>> Thank you for the hint. I add a new type
> >>>>>>>>>>>>> `UnknownStateStoreException`
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> this case.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Also, we should still have fatal exception
> >>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
> >>>>>> remove
> >>>>>>>>> it?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thank you point this, already add it again.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> The KIP already updated, please take a look.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks for updating the KIP!

One last comment/question: you kept `StateStoreNotAvailableException` in
favor of `StreamsNotRunningException` (to merge both as suggested).

I am wondering, if it might be better to keep
`StreamsNotRunningException` instead of
`StateStoreNotAvailableException`, because this exception is thrown if
Streams is in state PENDING_SHUTDOWN / NOT_RUNNING / ERROR ?



-Matthias

On 1/17/20 9:56 PM, John Roesler wrote:
> Thanks, Vito. I've just cast my vote.
> -John
> 
> On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
>> Hi, folks,
>>
>> Just update the KIP, please take a look.
>>
>> Thanks!
>>
>> ---
>> Vito
>>
>>
>> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng <vi...@is-land.com.tw> wrote:
>>
>>> Thanks Bill, John and Matthias. Glad you guys joined this discussion.
>>> I got a lot out of the discussion.
>>>
>>> I would like to update KIP-216 base on John's suggestion to remove the
>>> category.
>>>
>>>
>>> ---
>>> Vito
>>>
>>>
>>> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>>> Nevertheless, if we omit the categorization, it’s moot.
>>>>
>>>> Ack.
>>>>
>>>> I am fine to remove the middle tier. As John pointed out, it might be
>>>> weird to have only one concrete exception type per category. We can also
>>>> explain in detail how to handle each exception in their JavaDocs.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
>>>>> Vito,
>>>>>
>>>>> Thanks for the updates, the KIP LGTM.
>>>>>
>>>>> -Bill
>>>>>
>>>>> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org>
>>>> wrote:
>>>>>
>>>>>> Hi Vito,
>>>>>>
>>>>>> Haha, your archive game is on point!
>>>>>>
>>>>>> What Matthias said in that email is essentially what I figured was the
>>>>>> rationale. It makes sense, but the point I was making is that this
>>>> really
>>>>>> doesn’t seem like a good way to structure a production app. On the
>>>> other
>>>>>> hand, considering the exception fatal has a good chance of avoiding a
>>>>>> frustrating debug session if you just forgot to call start.
>>>>>>
>>>>>> Nevertheless, if we omit the categorization, it’s moot.
>>>>>>
>>>>>> It would be easy to add a categorization layer later if we want it, but
>>>>>> not very easy to change it if we get it wrong.
>>>>>>
>>>>>> Thanks for your consideration!
>>>>>> -John
>>>>>>
>>>>>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
>>>>>>> Hi John,
>>>>>>>
>>>>>>> About `StreamsNotStartedException is strange` --
>>>>>>> The original idea came from Matthias, two years ago. :)
>>>>>>> You can reference here:
>>>>>>>
>>>>>>
>>>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
>>>>>>>
>>>>>>> About omitting the categorization --
>>>>>>> It looks reasonable. I'm fine with omitting the categorization but not
>>>>>> very
>>>>>>> sure it is a good choice.
>>>>>>> Does any other folks provide opinion?
>>>>>>>
>>>>>>>
>>>>>>> Hi, folks,
>>>>>>>
>>>>>>> Just update the KIP-216, please take a look.
>>>>>>>
>>>>>>> ---
>>>>>>> Vito
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw>
>>>> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>> Hi, folks,
>>>>>>>>
>>>>>>>> Thank you suggestion, really appreciate it. :)
>>>>>>>> I understand your concern. I'll merge StreamsNotRunningException and
>>>>>>>> StateStoreNotAvailableException.
>>>>>>>>
>>>>>>>>
>>>>>>>> ---
>>>>>>>> Vito
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hey Vito,
>>>>>>>>>
>>>>>>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
>>>>>>>>> Hopefully, we can close it out soon.
>>>>>>>>>
>>>>>>>>> I agree we can drop StreamsNotRunningException in favor of
>>>>>>>>> just StateStoreNotAvailableException.
>>>>>>>>>
>>>>>>>>> Unfortunately, I have some higher-level concerns. The value
>>>>>>>>> of these exceptions is that they tell you how to handle the
>>>>>>>>> various situations that can arise while querying a distributed
>>>>>>>>> data store.
>>>>>>>>>
>>>>>>>>> Ideally, as a caller, I should be able to just catch "retriable" or
>>>>>>>>> "fatal" and handle them appropriately. Otherwise, there's no
>>>>>>>>> point in having categories, and we should just have all the
>>>>>>>>> exceptions extend InvalidStateStoreException.
>>>>>>>>>
>>>>>>>>> Presently, it's not possible to tell from just the
>>>>>>>>> "retriable"/"fatal" distinction what to do. You  can tell
>>>>>>>>> from the descriptions of the various exceptions. E.g.:
>>>>>>>>>
>>>>>>>>> Retriable:
>>>>>>>>>  * StreamsRebalancingException: the exact same call
>>>>>>>>>     should just be retried until the rebalance is complete
>>>>>>>>>  * StateStoreMigratedException: the store handle is
>>>>>>>>>     now invalid, so you need to re-discover the instance
>>>>>>>>>     and get a new handle on that instance. In other words,
>>>>>>>>>     the query itself may be valid, but the particular method
>>>>>>>>>     invocation on this particular instance has encountered
>>>>>>>>>     a fatal exception.
>>>>>>>>>
>>>>>>>>> Fatal:
>>>>>>>>>  * UnknownStateStoreException: this is truly fatal. No amount
>>>>>>>>>     of retrying or re-discovering is going to get you a handle on a
>>>>>>>>>     store that doesn't exist in the cluster.
>>>>>>>>>  * StateStoreNotAvailableException: this is actually recoverable,
>>>>>>>>>     since the store might exist in the cluster, but isn't available
>>>> on
>>>>>>>>>     this particular instance (which is shut down or whatever).
>>>>>>>>>
>>>>>>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
>>>>>>>>> with omitting the categorization and just having 5 subclasses
>>>>>>>>> of InvalidStateStoreException. Each of them would tell you
>>>>>>>>> how to handle them, and it's not too many to really
>>>>>>>>> understand and handle each one.
>>>>>>>>>
>>>>>>>>> If you really want to have a middle tier, I'd recommend:
>>>>>>>>> * RetryableStateStoreException: the exact same call
>>>>>>>>>     should be repeated.
>>>>>>>>> * RecoverableStateStoreException: the store handle
>>>>>>>>>     should be discarded and the caller should re-discover
>>>>>>>>>     the location of the store and repeat the query on the
>>>>>>>>>     correct instance.
>>>>>>>>> * FatalStateStoreException: the query/request is totally
>>>>>>>>>     invalid and will never succeed.
>>>>>>>>>
>>>>>>>>> However, attempting to categorize the proposed exceptions
>>>>>>>>> reveals even problems with this categorization:
>>>>>>>>> Retriable:
>>>>>>>>> * StreamsRebalancingException
>>>>>>>>> Recoverable:
>>>>>>>>> * StateStoreMigratedException
>>>>>>>>> * StreamsNotRunningException
>>>>>>>>> Fatal:
>>>>>>>>> * UnknownStateStoreException
>>>>>>>>>
>>>>>>>>> But StreamsNotStartedException is strange... It means that
>>>>>>>>> one code path got a handle on a specific KafkaStreams object
>>>>>>>>> instance and sent it a query before another code path
>>>>>>>>> invoked the start() method on the exact same object instance.
>>>>>>>>> It seems like the most likely scenario is that whoever wrote
>>>>>>>>> the program just forgot to call start() before querying, in
>>>>>>>>> which case, retrying isn't going to help, and a fatal exception
>>>>>>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
>>>>>>>>> experience" problem, and making it fatal would be more
>>>>>>>>> helpful. Even in a production context, there's no reason not
>>>>>>>>> to sequence your application startup such that you don't
>>>>>>>>> accept queries until after Streams is started. Thus, I guess
>>>>>>>>> I'd categorize it under "fatal".
>>>>>>>>>
>>>>>>>>> Regardless of whether you make it fatal or retriable, you'd
>>>>>>>>> still have a whole category with only one exception in it,
>>>>>>>>> and the other two categories only have two exceptions.
>>>>>>>>> Plus, as you pointed out in the KIP, you can't get all
>>>>>>>>> exceptions in all cases anyway:
>>>>>>>>> * store() can only throw NotStarted, NotRunning,
>>>>>>>>>     and Unknown
>>>>>>>>> * actual store queries can only throw Rebalancing,
>>>>>>>>>     Migrated, and NotRunning
>>>>>>>>>
>>>>>>>>> Thus, in practice also, there are exactly three categories
>>>>>>>>> and also exactly three exception types. It doesn't seem
>>>>>>>>> like there's a great advantage to the categories here. To
>>>>>>>>> avoid the categorization problem and also to clarify what
>>>>>>>>> exceptions can actually be thrown in different circumstances,
>>>>>>>>> it seems like we should just:
>>>>>>>>> * get rid of the middle tier and make all the exceptions
>>>>>>>>>     extend InvalidStateStoreException
>>>>>>>>> * drop StateStoreNotAvailableException in favor of
>>>>>>>>>     StreamsNotRunningException
>>>>>>>>> * clearly document on all public methods which exceptions
>>>>>>>>>     need to be handled
>>>>>>>>>
>>>>>>>>> How do you feel about this?
>>>>>>>>> Thanks,
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
>>>>>>>>>> Thanks for KIP Vito.
>>>>>>>>>>
>>>>>>>>>> Overall the KIP LGTM, but I'd have to agree with others on merging
>>>>>> the
>>>>>>>>>> `StreamsNotRunningException` and `StateStoreNotAvailableException`
>>>>>>>>> classes.
>>>>>>>>>>
>>>>>>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
>>>>>>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
>>>>>> when to
>>>>>>>>>> use the different
>>>>>>>>>> exceptions.  Maybe a good middle ground would be to have a detailed
>>>>>>>>>> exception message.
>>>>>>>>>>
>>>>>>>>>> The KIP freeze is close, so I think if we can agree on this, we can
>>>>>>>>> wrap up
>>>>>>>>>> the voting soon.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Bill
>>>>>>>>>>
>>>>>>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
>>>>>> matthias@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Vito,
>>>>>>>>>>>
>>>>>>>>>>> It's still unclear to me what the advantage is, to have both
>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>> `StateStoreNotAvailableException`?
>>>>>>>>>>>
>>>>>>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
>>>>>> ERROR`
>>>>>>>>>>> and thus, for a user point of view, why does it matter if the
>>>>>> store is
>>>>>>>>>>> closed on not? I don't understand why/how this information would
>>>>>> be
>>>>>>>>>>> useful? Do you have a concrete example in mind how a user would
>>>>>> react
>>>>>>>>>>> differently to both exceptions?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
>>>>>> best
>>>>>>>>> to
>>>>>>>>>>> actually do this on a per-query basis, ie, have an overload
>>>>>>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow to
>>>>>>>>>>> _disable_ the exception and opt-in to query a active store during
>>>>>>>>>>> recovery. However, as KIP-535 actually introduces this change in
>>>>>>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
>>>>>> should be
>>>>>>>>>>> updated. I'll follow up on the other KIP thread to raise this
>>>>>> point.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
>>>>>>>>>>>> Hi, Matthias & Vinoth,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>
>>>>>>>>>>>>> What is still unclear to me is, what we gain by having both
>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>> `StateStoreNotAvailableException`. Both
>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>>>>>> if the
>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway? Maybe
>>>>>> I
>>>>>>>>> miss
>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>
>>>>>>>>>>>> Yes, both `StreamsNotRunningException` and
>>>>>>>>>>>> `StateStoreNotAvailableException` are fatal exception.
>>>>>>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
>>>>>> state
>>>>>>>>>>> store
>>>>>>>>>>>> related.
>>>>>>>>>>>> I think it would be helpful that if user need to distinguish
>>>>>> these
>>>>>>>>> two
>>>>>>>>>>>> different case to handle it.
>>>>>>>>>>>>
>>>>>>>>>>>> I'm not very sure, does that make sense?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> ---
>>>>>>>>>>>> Vito
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
>>>>>> vinoth@apache.org>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> +1 on merging `StreamsNotRunningException` and
>>>>>>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
>>>>>>>>> anyway. IMO
>>>>>>>>>>>>> its best to have these exceptions be about the state store
>>>>>> (and not
>>>>>>>>>>> streams
>>>>>>>>>>>>> state), to easier understanding.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
>>>>>>>>> rebalancing
>>>>>>>>>>>>> state. So do we need the StreamsRebalancingException?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
>>>>>> matthias@confluent.io>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> Sorry that I dropped the ball on this...
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
>>>>>> start
>>>>>>>>> a
>>>>>>>>>>> VOTE
>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What is still unclear to me is, what we gain by having both
>>>>>>>>>>>>>> `StreamsNotRunningException` and
>>>>>>>>> `StateStoreNotAvailableException`.
>>>>>>>>>>> Both
>>>>>>>>>>>>>> exception are thrown when KafkaStreams is in state
>>>>>>>>> PENDING_SHUTDOWN /
>>>>>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>>>>>> if
>>>>>>>>> the
>>>>>>>>>>>>>> state store is closed on not -- I can't query it anyway?
>>>>>> Maybe I
>>>>>>>>> miss
>>>>>>>>>>>>>> something thought?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>>>>>>>>>>>>>>> Sorry for the late reply, thanks for the review.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
>>>>>>>>> might be
>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
>>>>>>>>> back to
>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>>>>>> store
>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>>>>>>>>> `RUNNING`
>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thank you point this, already updated.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Why do we need to distinguish between
>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
>>>>>>>>> reasons, I
>>>>>>>>>>>>> think
>>>>>>>>>>>>>>> it would be helpful that the
>>>>>>>>>>>>>>> user can distinguish whether it is caused by the state store
>>>>>>>>> closed.
>>>>>>>>>>>>>>> (Maybe I am wrong...)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
>>>>>> and
>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
>>>>>> the
>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>> because that is the level of granularity in which we
>>>>>>>>> enable/disable
>>>>>>>>>>>>> IQ atm.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Totally agree. Do you mean the naming of state store
>>>>>> exceptions?
>>>>>>>>>>>>>>> I don't have special reason to distinguish these two.
>>>>>>>>>>>>>>> Your suggestion look more reasonable for the exception
>>>>>> naming.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
>>>>>> user
>>>>>>>>> need
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
>>>>>>>>> handle is
>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
>>>>>>>>>>> difference
>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>>>>>> "blind"
>>>>>>>>> retries
>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
>>>>>> instance
>>>>>>>>> after
>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Nice, it's great! Thank you.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The KIP already updated, please take a look. :)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Any update on this KIP?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
>>>>>> busy.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> About `StateStoreMigratedException`:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
>>>>>>>>> might be
>>>>>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
>>>>>>>>> back to
>>>>>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>>>>>> store
>>>>>>>>>>>>> handle.
>>>>>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>>>>>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>>>>>>>>> `RUNNING`
>>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Why do we need to distinguish between
>>>>>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
>>>>>> instance and
>>>>>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
>>>>>> the
>>>>>>>>>>>>> instance,
>>>>>>>>>>>>>>>>> because that is the level of granularity in which we
>>>>>>>>> enable/disable
>>>>>>>>>>>>> IQ
>>>>>>>>>>>>>>>> atm.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
>>>>>>>>> user
>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
>>>>>>>>> handle is
>>>>>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>>>>>> "blind"
>>>>>>>>>>> retries
>>>>>>>>>>>>>>>>> that either resolve (if the store is still on the same
>>>>>> instance
>>>>>>>>>>> after
>>>>>>>>>>>>>>>>> rebalancing finishes, or changes to
>>>>>>>>> `StateStoreMigratedException` if
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> store was migrated away during rebalancing).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>>>>>>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>>>> <https://shorturl.at/CDNT9>
>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Please use the following instead:
>>>>>> https://shorturl.at/bkKQU
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ---
>>>>>>>>>>>>>>>>>> Vito
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
>>>>>>>>> vito@is-land.com.tw>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks, Matthias!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
>>>>>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>>>>>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
>>>>>>>>> handed out
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> store handle. We may throw
>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> `StateStoreMigratedException`.
>>>>>>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
>>>>>> not
>>>>>>>>>>>>> running(
>>>>>>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
>>>>>>>>> `StateStoreMigratedException`
>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
>>>>>> we
>>>>>>>>> do not
>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
>>>>>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I understand your point. I rename
>>>>>>>>>>>>> `StreamThreadNotRunningException` to
>>>>>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> About check unknown state store names:
>>>>>>>>>>>>>>>>>>> Thank you for the hint. I add a new type
>>>>>>>>>>>>> `UnknownStateStoreException`
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> this case.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Also, we should still have fatal exception
>>>>>>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
>>>>>> remove
>>>>>>>>> it?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thank you point this, already add it again.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The KIP already updated, please take a look.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ---
>>>>>>>>>>>>>>>>>>> Vito
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by John Roesler <vv...@apache.org>.
Thanks, Vito. I've just cast my vote.
-John

On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote:
> Hi, folks,
> 
> Just update the KIP, please take a look.
> 
> Thanks!
> 
> ---
> Vito
> 
> 
> On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> 
> > Thanks Bill, John and Matthias. Glad you guys joined this discussion.
> > I got a lot out of the discussion.
> >
> > I would like to update KIP-216 base on John's suggestion to remove the
> > category.
> >
> >
> > ---
> > Vito
> >
> >
> > On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> > Nevertheless, if we omit the categorization, it’s moot.
> >>
> >> Ack.
> >>
> >> I am fine to remove the middle tier. As John pointed out, it might be
> >> weird to have only one concrete exception type per category. We can also
> >> explain in detail how to handle each exception in their JavaDocs.
> >>
> >>
> >> -Matthias
> >>
> >> On 1/16/20 6:38 AM, Bill Bejeck wrote:
> >> > Vito,
> >> >
> >> > Thanks for the updates, the KIP LGTM.
> >> >
> >> > -Bill
> >> >
> >> > On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org>
> >> wrote:
> >> >
> >> >> Hi Vito,
> >> >>
> >> >> Haha, your archive game is on point!
> >> >>
> >> >> What Matthias said in that email is essentially what I figured was the
> >> >> rationale. It makes sense, but the point I was making is that this
> >> really
> >> >> doesn’t seem like a good way to structure a production app. On the
> >> other
> >> >> hand, considering the exception fatal has a good chance of avoiding a
> >> >> frustrating debug session if you just forgot to call start.
> >> >>
> >> >> Nevertheless, if we omit the categorization, it’s moot.
> >> >>
> >> >> It would be easy to add a categorization layer later if we want it, but
> >> >> not very easy to change it if we get it wrong.
> >> >>
> >> >> Thanks for your consideration!
> >> >> -John
> >> >>
> >> >> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
> >> >>> Hi John,
> >> >>>
> >> >>> About `StreamsNotStartedException is strange` --
> >> >>> The original idea came from Matthias, two years ago. :)
> >> >>> You can reference here:
> >> >>>
> >> >>
> >> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> >> >>>
> >> >>> About omitting the categorization --
> >> >>> It looks reasonable. I'm fine with omitting the categorization but not
> >> >> very
> >> >>> sure it is a good choice.
> >> >>> Does any other folks provide opinion?
> >> >>>
> >> >>>
> >> >>> Hi, folks,
> >> >>>
> >> >>> Just update the KIP-216, please take a look.
> >> >>>
> >> >>> ---
> >> >>> Vito
> >> >>>
> >> >>>
> >> >>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw>
> >> wrote:
> >> >>>
> >> >>>>
> >> >>>> Hi, folks,
> >> >>>>
> >> >>>> Thank you suggestion, really appreciate it. :)
> >> >>>> I understand your concern. I'll merge StreamsNotRunningException and
> >> >>>> StateStoreNotAvailableException.
> >> >>>>
> >> >>>>
> >> >>>> ---
> >> >>>> Vito
> >> >>>>
> >> >>>>
> >> >>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org>
> >> >> wrote:
> >> >>>>
> >> >>>>> Hey Vito,
> >> >>>>>
> >> >>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
> >> >>>>> Hopefully, we can close it out soon.
> >> >>>>>
> >> >>>>> I agree we can drop StreamsNotRunningException in favor of
> >> >>>>> just StateStoreNotAvailableException.
> >> >>>>>
> >> >>>>> Unfortunately, I have some higher-level concerns. The value
> >> >>>>> of these exceptions is that they tell you how to handle the
> >> >>>>> various situations that can arise while querying a distributed
> >> >>>>> data store.
> >> >>>>>
> >> >>>>> Ideally, as a caller, I should be able to just catch "retriable" or
> >> >>>>> "fatal" and handle them appropriately. Otherwise, there's no
> >> >>>>> point in having categories, and we should just have all the
> >> >>>>> exceptions extend InvalidStateStoreException.
> >> >>>>>
> >> >>>>> Presently, it's not possible to tell from just the
> >> >>>>> "retriable"/"fatal" distinction what to do. You  can tell
> >> >>>>> from the descriptions of the various exceptions. E.g.:
> >> >>>>>
> >> >>>>> Retriable:
> >> >>>>>  * StreamsRebalancingException: the exact same call
> >> >>>>>     should just be retried until the rebalance is complete
> >> >>>>>  * StateStoreMigratedException: the store handle is
> >> >>>>>     now invalid, so you need to re-discover the instance
> >> >>>>>     and get a new handle on that instance. In other words,
> >> >>>>>     the query itself may be valid, but the particular method
> >> >>>>>     invocation on this particular instance has encountered
> >> >>>>>     a fatal exception.
> >> >>>>>
> >> >>>>> Fatal:
> >> >>>>>  * UnknownStateStoreException: this is truly fatal. No amount
> >> >>>>>     of retrying or re-discovering is going to get you a handle on a
> >> >>>>>     store that doesn't exist in the cluster.
> >> >>>>>  * StateStoreNotAvailableException: this is actually recoverable,
> >> >>>>>     since the store might exist in the cluster, but isn't available
> >> on
> >> >>>>>     this particular instance (which is shut down or whatever).
> >> >>>>>
> >> >>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> >> >>>>> with omitting the categorization and just having 5 subclasses
> >> >>>>> of InvalidStateStoreException. Each of them would tell you
> >> >>>>> how to handle them, and it's not too many to really
> >> >>>>> understand and handle each one.
> >> >>>>>
> >> >>>>> If you really want to have a middle tier, I'd recommend:
> >> >>>>> * RetryableStateStoreException: the exact same call
> >> >>>>>     should be repeated.
> >> >>>>> * RecoverableStateStoreException: the store handle
> >> >>>>>     should be discarded and the caller should re-discover
> >> >>>>>     the location of the store and repeat the query on the
> >> >>>>>     correct instance.
> >> >>>>> * FatalStateStoreException: the query/request is totally
> >> >>>>>     invalid and will never succeed.
> >> >>>>>
> >> >>>>> However, attempting to categorize the proposed exceptions
> >> >>>>> reveals even problems with this categorization:
> >> >>>>> Retriable:
> >> >>>>> * StreamsRebalancingException
> >> >>>>> Recoverable:
> >> >>>>> * StateStoreMigratedException
> >> >>>>> * StreamsNotRunningException
> >> >>>>> Fatal:
> >> >>>>> * UnknownStateStoreException
> >> >>>>>
> >> >>>>> But StreamsNotStartedException is strange... It means that
> >> >>>>> one code path got a handle on a specific KafkaStreams object
> >> >>>>> instance and sent it a query before another code path
> >> >>>>> invoked the start() method on the exact same object instance.
> >> >>>>> It seems like the most likely scenario is that whoever wrote
> >> >>>>> the program just forgot to call start() before querying, in
> >> >>>>> which case, retrying isn't going to help, and a fatal exception
> >> >>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
> >> >>>>> experience" problem, and making it fatal would be more
> >> >>>>> helpful. Even in a production context, there's no reason not
> >> >>>>> to sequence your application startup such that you don't
> >> >>>>> accept queries until after Streams is started. Thus, I guess
> >> >>>>> I'd categorize it under "fatal".
> >> >>>>>
> >> >>>>> Regardless of whether you make it fatal or retriable, you'd
> >> >>>>> still have a whole category with only one exception in it,
> >> >>>>> and the other two categories only have two exceptions.
> >> >>>>> Plus, as you pointed out in the KIP, you can't get all
> >> >>>>> exceptions in all cases anyway:
> >> >>>>> * store() can only throw NotStarted, NotRunning,
> >> >>>>>     and Unknown
> >> >>>>> * actual store queries can only throw Rebalancing,
> >> >>>>>     Migrated, and NotRunning
> >> >>>>>
> >> >>>>> Thus, in practice also, there are exactly three categories
> >> >>>>> and also exactly three exception types. It doesn't seem
> >> >>>>> like there's a great advantage to the categories here. To
> >> >>>>> avoid the categorization problem and also to clarify what
> >> >>>>> exceptions can actually be thrown in different circumstances,
> >> >>>>> it seems like we should just:
> >> >>>>> * get rid of the middle tier and make all the exceptions
> >> >>>>>     extend InvalidStateStoreException
> >> >>>>> * drop StateStoreNotAvailableException in favor of
> >> >>>>>     StreamsNotRunningException
> >> >>>>> * clearly document on all public methods which exceptions
> >> >>>>>     need to be handled
> >> >>>>>
> >> >>>>> How do you feel about this?
> >> >>>>> Thanks,
> >> >>>>> -John
> >> >>>>>
> >> >>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> >> >>>>>> Thanks for KIP Vito.
> >> >>>>>>
> >> >>>>>> Overall the KIP LGTM, but I'd have to agree with others on merging
> >> >> the
> >> >>>>>> `StreamsNotRunningException` and `StateStoreNotAvailableException`
> >> >>>>> classes.
> >> >>>>>>
> >> >>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> >> >>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
> >> >> when to
> >> >>>>>> use the different
> >> >>>>>> exceptions.  Maybe a good middle ground would be to have a detailed
> >> >>>>>> exception message.
> >> >>>>>>
> >> >>>>>> The KIP freeze is close, so I think if we can agree on this, we can
> >> >>>>> wrap up
> >> >>>>>> the voting soon.
> >> >>>>>>
> >> >>>>>> Thanks,
> >> >>>>>> Bill
> >> >>>>>>
> >> >>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
> >> >> matthias@confluent.io>
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>>> Vito,
> >> >>>>>>>
> >> >>>>>>> It's still unclear to me what the advantage is, to have both
> >> >>>>>>> `StreamsNotRunningException` and
> >> >> `StateStoreNotAvailableException`?
> >> >>>>>>>
> >> >>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
> >> >> ERROR`
> >> >>>>>>> and thus, for a user point of view, why does it matter if the
> >> >> store is
> >> >>>>>>> closed on not? I don't understand why/how this information would
> >> >> be
> >> >>>>>>> useful? Do you have a concrete example in mind how a user would
> >> >> react
> >> >>>>>>> differently to both exceptions?
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
> >> >> best
> >> >>>>> to
> >> >>>>>>> actually do this on a per-query basis, ie, have an overload
> >> >>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow to
> >> >>>>>>> _disable_ the exception and opt-in to query a active store during
> >> >>>>>>> recovery. However, as KIP-535 actually introduces this change in
> >> >>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
> >> >> should be
> >> >>>>>>> updated. I'll follow up on the other KIP thread to raise this
> >> >> point.
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> -Matthias
> >> >>>>>>>
> >> >>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
> >> >>>>>>>> Hi, Matthias & Vinoth,
> >> >>>>>>>>
> >> >>>>>>>> Thanks for the feedback.
> >> >>>>>>>>
> >> >>>>>>>>> What is still unclear to me is, what we gain by having both
> >> >>>>>>>>> `StreamsNotRunningException` and
> >> >>>>> `StateStoreNotAvailableException`. Both
> >> >>>>>>>>> exception are thrown when KafkaStreams is in state
> >> >>>>> PENDING_SHUTDOWN /
> >> >>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> >> >> if the
> >> >>>>>>>>> state store is closed on not -- I can't query it anyway? Maybe
> >> >> I
> >> >>>>> miss
> >> >>>>>>>>> something thought?
> >> >>>>>>>>
> >> >>>>>>>> Yes, both `StreamsNotRunningException` and
> >> >>>>>>>> `StateStoreNotAvailableException` are fatal exception.
> >> >>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
> >> >> state
> >> >>>>>>> store
> >> >>>>>>>> related.
> >> >>>>>>>> I think it would be helpful that if user need to distinguish
> >> >> these
> >> >>>>> two
> >> >>>>>>>> different case to handle it.
> >> >>>>>>>>
> >> >>>>>>>> I'm not very sure, does that make sense?
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> ---
> >> >>>>>>>> Vito
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
> >> >> vinoth@apache.org>
> >> >>>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>>> +1 on merging `StreamsNotRunningException` and
> >> >>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
> >> >>>>> anyway. IMO
> >> >>>>>>>>> its best to have these exceptions be about the state store
> >> >> (and not
> >> >>>>>>> streams
> >> >>>>>>>>> state), to easier understanding.
> >> >>>>>>>>>
> >> >>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
> >> >>>>> rebalancing
> >> >>>>>>>>> state. So do we need the StreamsRebalancingException?
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
> >> >> matthias@confluent.io>
> >> >>>>>>> wrote:
> >> >>>>>>>>>> Sorry that I dropped the ball on this...
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
> >> >> start
> >> >>>>> a
> >> >>>>>>> VOTE
> >> >>>>>>>>>> thread.
> >> >>>>>>>>>>
> >> >>>>>>>>>> What is still unclear to me is, what we gain by having both
> >> >>>>>>>>>> `StreamsNotRunningException` and
> >> >>>>> `StateStoreNotAvailableException`.
> >> >>>>>>> Both
> >> >>>>>>>>>> exception are thrown when KafkaStreams is in state
> >> >>>>> PENDING_SHUTDOWN /
> >> >>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> >> >> if
> >> >>>>> the
> >> >>>>>>>>>> state store is closed on not -- I can't query it anyway?
> >> >> Maybe I
> >> >>>>> miss
> >> >>>>>>>>>> something thought?
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> -Matthias
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> >> >>>>>>>>>>> Sorry for the late reply, thanks for the review.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>> About `StateStoreMigratedException`:
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
> >> >>>>> might be
> >> >>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
> >> >>>>> back to
> >> >>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> >> >> store
> >> >>>>>>> handle.
> >> >>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >> >>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >> >>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> >> >>>>> `RUNNING`
> >> >>>>>>>>> state.
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Thank you point this, already updated.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Why do we need to distinguish between
> >> >>>>>>> `KafkaStreamsNotRunningException`
> >> >>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
> >> >>>>> reasons, I
> >> >>>>>>>>> think
> >> >>>>>>>>>>> it would be helpful that the
> >> >>>>>>>>>>> user can distinguish whether it is caused by the state store
> >> >>>>> closed.
> >> >>>>>>>>>>> (Maybe I am wrong...)
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
> >> >> and
> >> >>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
> >> >> the
> >> >>>>>>>>> instance,
> >> >>>>>>>>>>>> because that is the level of granularity in which we
> >> >>>>> enable/disable
> >> >>>>>>>>> IQ atm.
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Totally agree. Do you mean the naming of state store
> >> >> exceptions?
> >> >>>>>>>>>>> I don't have special reason to distinguish these two.
> >> >>>>>>>>>>> Your suggestion look more reasonable for the exception
> >> >> naming.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
> >> >> user
> >> >>>>> need
> >> >>>>>>>>> to
> >> >>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
> >> >>>>> handle is
> >> >>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
> >> >>>>>>> difference
> >> >>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >> >> "blind"
> >> >>>>> retries
> >> >>>>>>>>>>>> that either resolve (if the store is still on the same
> >> >> instance
> >> >>>>> after
> >> >>>>>>>>>>>> rebalancing finishes, or changes to
> >> >>>>> `StateStoreMigratedException` if
> >> >>>>>>>>> the
> >> >>>>>>>>>>>> store was migrated away during rebalancing).
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Nice, it's great! Thank you.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> The KIP already updated, please take a look. :)
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> >> >>>>>>> matthias@confluent.io
> >> >>>>>>>>>>
> >> >>>>>>>>>>> wrote:
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>> Any update on this KIP?
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >> >>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
> >> >> busy.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> About `StateStoreMigratedException`:
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
> >> >>>>> might be
> >> >>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
> >> >>>>> back to
> >> >>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> >> >> store
> >> >>>>>>>>> handle.
> >> >>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >> >>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >> >>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> >> >>>>> `RUNNING`
> >> >>>>>>>>>>>> state.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> Why do we need to distinguish between
> >> >>>>>>>>> `KafkaStreamsNotRunningException`
> >> >>>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
> >> >> instance and
> >> >>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
> >> >> the
> >> >>>>>>>>> instance,
> >> >>>>>>>>>>>>> because that is the level of granularity in which we
> >> >>>>> enable/disable
> >> >>>>>>>>> IQ
> >> >>>>>>>>>>>> atm.
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
> >> >>>>> user
> >> >>>>>>>>> need to
> >> >>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
> >> >>>>> handle is
> >> >>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
> >> >>>>>>>>> difference
> >> >>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >> >> "blind"
> >> >>>>>>> retries
> >> >>>>>>>>>>>>> that either resolve (if the store is still on the same
> >> >> instance
> >> >>>>>>> after
> >> >>>>>>>>>>>>> rebalancing finishes, or changes to
> >> >>>>> `StateStoreMigratedException` if
> >> >>>>>>>>> the
> >> >>>>>>>>>>>>> store was migrated away during rebalancing).
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> -Matthias
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >> >>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> >> <https://shorturl.at/CDNT9>
> >> >> <https://shorturl.at/CDNT9>
> >> >>>>> <https://shorturl.at/CDNT9>
> >> >>>>>>> <https://shorturl.at/CDNT9>
> >> >>>>>>>>> <https://shorturl.at/CDNT9>
> >> >>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >> >>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>> Please use the following instead:
> >> >> https://shorturl.at/bkKQU
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>> ---
> >> >>>>>>>>>>>>>> Vito
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
> >> >>>>> vito@is-land.com.tw>
> >> >>>>>>>>> wrote:
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> Thanks, Matthias!
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
> >> >>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >> >>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
> >> >>>>> handed out
> >> >>>>>>>>> the
> >> >>>>>>>>>>>>>>> store handle. We may throw
> >> >> `KafkaStreamsNotRunningException`
> >> >>>>> or
> >> >>>>>>>>>>>>>>> `StateStoreMigratedException`.
> >> >>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> >> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
> >> >> not
> >> >>>>>>>>> running(
> >> >>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
> >> >>>>> `StateStoreMigratedException`
> >> >>>>>>>>> when
> >> >>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
> >> >> we
> >> >>>>> do not
> >> >>>>>>>>> need
> >> >>>>>>>>>>>> to
> >> >>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
> >> >>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> I understand your point. I rename
> >> >>>>>>>>> `StreamThreadNotRunningException` to
> >> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> About check unknown state store names:
> >> >>>>>>>>>>>>>>> Thank you for the hint. I add a new type
> >> >>>>>>>>> `UnknownStateStoreException`
> >> >>>>>>>>>>>> for
> >> >>>>>>>>>>>>>>> this case.
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>>> Also, we should still have fatal exception
> >> >>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
> >> >> remove
> >> >>>>> it?
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> Thank you point this, already add it again.
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> The KIP already updated, please take a look.
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>> ---
> >> >>>>>>>>>>>>>>> Vito
> >> >>>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>
> >> >>
> >> >
> >>
> >>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Hi, folks,

Just update the KIP, please take a look.

Thanks!

---
Vito


On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng <vi...@is-land.com.tw> wrote:

> Thanks Bill, John and Matthias. Glad you guys joined this discussion.
> I got a lot out of the discussion.
>
> I would like to update KIP-216 base on John's suggestion to remove the
> category.
>
>
> ---
> Vito
>
>
> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> > Nevertheless, if we omit the categorization, it’s moot.
>>
>> Ack.
>>
>> I am fine to remove the middle tier. As John pointed out, it might be
>> weird to have only one concrete exception type per category. We can also
>> explain in detail how to handle each exception in their JavaDocs.
>>
>>
>> -Matthias
>>
>> On 1/16/20 6:38 AM, Bill Bejeck wrote:
>> > Vito,
>> >
>> > Thanks for the updates, the KIP LGTM.
>> >
>> > -Bill
>> >
>> > On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org>
>> wrote:
>> >
>> >> Hi Vito,
>> >>
>> >> Haha, your archive game is on point!
>> >>
>> >> What Matthias said in that email is essentially what I figured was the
>> >> rationale. It makes sense, but the point I was making is that this
>> really
>> >> doesn’t seem like a good way to structure a production app. On the
>> other
>> >> hand, considering the exception fatal has a good chance of avoiding a
>> >> frustrating debug session if you just forgot to call start.
>> >>
>> >> Nevertheless, if we omit the categorization, it’s moot.
>> >>
>> >> It would be easy to add a categorization layer later if we want it, but
>> >> not very easy to change it if we get it wrong.
>> >>
>> >> Thanks for your consideration!
>> >> -John
>> >>
>> >> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
>> >>> Hi John,
>> >>>
>> >>> About `StreamsNotStartedException is strange` --
>> >>> The original idea came from Matthias, two years ago. :)
>> >>> You can reference here:
>> >>>
>> >>
>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
>> >>>
>> >>> About omitting the categorization --
>> >>> It looks reasonable. I'm fine with omitting the categorization but not
>> >> very
>> >>> sure it is a good choice.
>> >>> Does any other folks provide opinion?
>> >>>
>> >>>
>> >>> Hi, folks,
>> >>>
>> >>> Just update the KIP-216, please take a look.
>> >>>
>> >>> ---
>> >>> Vito
>> >>>
>> >>>
>> >>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw>
>> wrote:
>> >>>
>> >>>>
>> >>>> Hi, folks,
>> >>>>
>> >>>> Thank you suggestion, really appreciate it. :)
>> >>>> I understand your concern. I'll merge StreamsNotRunningException and
>> >>>> StateStoreNotAvailableException.
>> >>>>
>> >>>>
>> >>>> ---
>> >>>> Vito
>> >>>>
>> >>>>
>> >>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org>
>> >> wrote:
>> >>>>
>> >>>>> Hey Vito,
>> >>>>>
>> >>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
>> >>>>> Hopefully, we can close it out soon.
>> >>>>>
>> >>>>> I agree we can drop StreamsNotRunningException in favor of
>> >>>>> just StateStoreNotAvailableException.
>> >>>>>
>> >>>>> Unfortunately, I have some higher-level concerns. The value
>> >>>>> of these exceptions is that they tell you how to handle the
>> >>>>> various situations that can arise while querying a distributed
>> >>>>> data store.
>> >>>>>
>> >>>>> Ideally, as a caller, I should be able to just catch "retriable" or
>> >>>>> "fatal" and handle them appropriately. Otherwise, there's no
>> >>>>> point in having categories, and we should just have all the
>> >>>>> exceptions extend InvalidStateStoreException.
>> >>>>>
>> >>>>> Presently, it's not possible to tell from just the
>> >>>>> "retriable"/"fatal" distinction what to do. You  can tell
>> >>>>> from the descriptions of the various exceptions. E.g.:
>> >>>>>
>> >>>>> Retriable:
>> >>>>>  * StreamsRebalancingException: the exact same call
>> >>>>>     should just be retried until the rebalance is complete
>> >>>>>  * StateStoreMigratedException: the store handle is
>> >>>>>     now invalid, so you need to re-discover the instance
>> >>>>>     and get a new handle on that instance. In other words,
>> >>>>>     the query itself may be valid, but the particular method
>> >>>>>     invocation on this particular instance has encountered
>> >>>>>     a fatal exception.
>> >>>>>
>> >>>>> Fatal:
>> >>>>>  * UnknownStateStoreException: this is truly fatal. No amount
>> >>>>>     of retrying or re-discovering is going to get you a handle on a
>> >>>>>     store that doesn't exist in the cluster.
>> >>>>>  * StateStoreNotAvailableException: this is actually recoverable,
>> >>>>>     since the store might exist in the cluster, but isn't available
>> on
>> >>>>>     this particular instance (which is shut down or whatever).
>> >>>>>
>> >>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
>> >>>>> with omitting the categorization and just having 5 subclasses
>> >>>>> of InvalidStateStoreException. Each of them would tell you
>> >>>>> how to handle them, and it's not too many to really
>> >>>>> understand and handle each one.
>> >>>>>
>> >>>>> If you really want to have a middle tier, I'd recommend:
>> >>>>> * RetryableStateStoreException: the exact same call
>> >>>>>     should be repeated.
>> >>>>> * RecoverableStateStoreException: the store handle
>> >>>>>     should be discarded and the caller should re-discover
>> >>>>>     the location of the store and repeat the query on the
>> >>>>>     correct instance.
>> >>>>> * FatalStateStoreException: the query/request is totally
>> >>>>>     invalid and will never succeed.
>> >>>>>
>> >>>>> However, attempting to categorize the proposed exceptions
>> >>>>> reveals even problems with this categorization:
>> >>>>> Retriable:
>> >>>>> * StreamsRebalancingException
>> >>>>> Recoverable:
>> >>>>> * StateStoreMigratedException
>> >>>>> * StreamsNotRunningException
>> >>>>> Fatal:
>> >>>>> * UnknownStateStoreException
>> >>>>>
>> >>>>> But StreamsNotStartedException is strange... It means that
>> >>>>> one code path got a handle on a specific KafkaStreams object
>> >>>>> instance and sent it a query before another code path
>> >>>>> invoked the start() method on the exact same object instance.
>> >>>>> It seems like the most likely scenario is that whoever wrote
>> >>>>> the program just forgot to call start() before querying, in
>> >>>>> which case, retrying isn't going to help, and a fatal exception
>> >>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
>> >>>>> experience" problem, and making it fatal would be more
>> >>>>> helpful. Even in a production context, there's no reason not
>> >>>>> to sequence your application startup such that you don't
>> >>>>> accept queries until after Streams is started. Thus, I guess
>> >>>>> I'd categorize it under "fatal".
>> >>>>>
>> >>>>> Regardless of whether you make it fatal or retriable, you'd
>> >>>>> still have a whole category with only one exception in it,
>> >>>>> and the other two categories only have two exceptions.
>> >>>>> Plus, as you pointed out in the KIP, you can't get all
>> >>>>> exceptions in all cases anyway:
>> >>>>> * store() can only throw NotStarted, NotRunning,
>> >>>>>     and Unknown
>> >>>>> * actual store queries can only throw Rebalancing,
>> >>>>>     Migrated, and NotRunning
>> >>>>>
>> >>>>> Thus, in practice also, there are exactly three categories
>> >>>>> and also exactly three exception types. It doesn't seem
>> >>>>> like there's a great advantage to the categories here. To
>> >>>>> avoid the categorization problem and also to clarify what
>> >>>>> exceptions can actually be thrown in different circumstances,
>> >>>>> it seems like we should just:
>> >>>>> * get rid of the middle tier and make all the exceptions
>> >>>>>     extend InvalidStateStoreException
>> >>>>> * drop StateStoreNotAvailableException in favor of
>> >>>>>     StreamsNotRunningException
>> >>>>> * clearly document on all public methods which exceptions
>> >>>>>     need to be handled
>> >>>>>
>> >>>>> How do you feel about this?
>> >>>>> Thanks,
>> >>>>> -John
>> >>>>>
>> >>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
>> >>>>>> Thanks for KIP Vito.
>> >>>>>>
>> >>>>>> Overall the KIP LGTM, but I'd have to agree with others on merging
>> >> the
>> >>>>>> `StreamsNotRunningException` and `StateStoreNotAvailableException`
>> >>>>> classes.
>> >>>>>>
>> >>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
>> >>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
>> >> when to
>> >>>>>> use the different
>> >>>>>> exceptions.  Maybe a good middle ground would be to have a detailed
>> >>>>>> exception message.
>> >>>>>>
>> >>>>>> The KIP freeze is close, so I think if we can agree on this, we can
>> >>>>> wrap up
>> >>>>>> the voting soon.
>> >>>>>>
>> >>>>>> Thanks,
>> >>>>>> Bill
>> >>>>>>
>> >>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
>> >> matthias@confluent.io>
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>>> Vito,
>> >>>>>>>
>> >>>>>>> It's still unclear to me what the advantage is, to have both
>> >>>>>>> `StreamsNotRunningException` and
>> >> `StateStoreNotAvailableException`?
>> >>>>>>>
>> >>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
>> >> ERROR`
>> >>>>>>> and thus, for a user point of view, why does it matter if the
>> >> store is
>> >>>>>>> closed on not? I don't understand why/how this information would
>> >> be
>> >>>>>>> useful? Do you have a concrete example in mind how a user would
>> >> react
>> >>>>>>> differently to both exceptions?
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
>> >> best
>> >>>>> to
>> >>>>>>> actually do this on a per-query basis, ie, have an overload
>> >>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow to
>> >>>>>>> _disable_ the exception and opt-in to query a active store during
>> >>>>>>> recovery. However, as KIP-535 actually introduces this change in
>> >>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
>> >> should be
>> >>>>>>> updated. I'll follow up on the other KIP thread to raise this
>> >> point.
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> -Matthias
>> >>>>>>>
>> >>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
>> >>>>>>>> Hi, Matthias & Vinoth,
>> >>>>>>>>
>> >>>>>>>> Thanks for the feedback.
>> >>>>>>>>
>> >>>>>>>>> What is still unclear to me is, what we gain by having both
>> >>>>>>>>> `StreamsNotRunningException` and
>> >>>>> `StateStoreNotAvailableException`. Both
>> >>>>>>>>> exception are thrown when KafkaStreams is in state
>> >>>>> PENDING_SHUTDOWN /
>> >>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>> >> if the
>> >>>>>>>>> state store is closed on not -- I can't query it anyway? Maybe
>> >> I
>> >>>>> miss
>> >>>>>>>>> something thought?
>> >>>>>>>>
>> >>>>>>>> Yes, both `StreamsNotRunningException` and
>> >>>>>>>> `StateStoreNotAvailableException` are fatal exception.
>> >>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
>> >> state
>> >>>>>>> store
>> >>>>>>>> related.
>> >>>>>>>> I think it would be helpful that if user need to distinguish
>> >> these
>> >>>>> two
>> >>>>>>>> different case to handle it.
>> >>>>>>>>
>> >>>>>>>> I'm not very sure, does that make sense?
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> ---
>> >>>>>>>> Vito
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
>> >> vinoth@apache.org>
>> >>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>> +1 on merging `StreamsNotRunningException` and
>> >>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
>> >>>>> anyway. IMO
>> >>>>>>>>> its best to have these exceptions be about the state store
>> >> (and not
>> >>>>>>> streams
>> >>>>>>>>> state), to easier understanding.
>> >>>>>>>>>
>> >>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
>> >>>>> rebalancing
>> >>>>>>>>> state. So do we need the StreamsRebalancingException?
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
>> >> matthias@confluent.io>
>> >>>>>>> wrote:
>> >>>>>>>>>> Sorry that I dropped the ball on this...
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
>> >> start
>> >>>>> a
>> >>>>>>> VOTE
>> >>>>>>>>>> thread.
>> >>>>>>>>>>
>> >>>>>>>>>> What is still unclear to me is, what we gain by having both
>> >>>>>>>>>> `StreamsNotRunningException` and
>> >>>>> `StateStoreNotAvailableException`.
>> >>>>>>> Both
>> >>>>>>>>>> exception are thrown when KafkaStreams is in state
>> >>>>> PENDING_SHUTDOWN /
>> >>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>> >> if
>> >>>>> the
>> >>>>>>>>>> state store is closed on not -- I can't query it anyway?
>> >> Maybe I
>> >>>>> miss
>> >>>>>>>>>> something thought?
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> -Matthias
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>> >>>>>>>>>>> Sorry for the late reply, thanks for the review.
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>>> About `StateStoreMigratedException`:
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
>> >>>>> might be
>> >>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
>> >>>>> back to
>> >>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>> >> store
>> >>>>>>> handle.
>> >>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>> >>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>> >>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>> >>>>> `RUNNING`
>> >>>>>>>>> state.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> Thank you point this, already updated.
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> Why do we need to distinguish between
>> >>>>>>> `KafkaStreamsNotRunningException`
>> >>>>>>>>>>>> and `StateStoreNotAvailableException`?
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
>> >>>>> reasons, I
>> >>>>>>>>> think
>> >>>>>>>>>>> it would be helpful that the
>> >>>>>>>>>>> user can distinguish whether it is caused by the state store
>> >>>>> closed.
>> >>>>>>>>>>> (Maybe I am wrong...)
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
>> >> and
>> >>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
>> >> the
>> >>>>>>>>> instance,
>> >>>>>>>>>>>> because that is the level of granularity in which we
>> >>>>> enable/disable
>> >>>>>>>>> IQ atm.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> Totally agree. Do you mean the naming of state store
>> >> exceptions?
>> >>>>>>>>>>> I don't have special reason to distinguish these two.
>> >>>>>>>>>>> Your suggestion look more reasonable for the exception
>> >> naming.
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
>> >> user
>> >>>>> need
>> >>>>>>>>> to
>> >>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
>> >>>>> handle is
>> >>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
>> >>>>>>> difference
>> >>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>> >> "blind"
>> >>>>> retries
>> >>>>>>>>>>>> that either resolve (if the store is still on the same
>> >> instance
>> >>>>> after
>> >>>>>>>>>>>> rebalancing finishes, or changes to
>> >>>>> `StateStoreMigratedException` if
>> >>>>>>>>> the
>> >>>>>>>>>>>> store was migrated away during rebalancing).
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> Nice, it's great! Thank you.
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> The KIP already updated, please take a look. :)
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
>> >>>>>>> matthias@confluent.io
>> >>>>>>>>>>
>> >>>>>>>>>>> wrote:
>> >>>>>>>>>>>
>> >>>>>>>>>>>> Any update on this KIP?
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>> >>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
>> >> busy.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> About `StateStoreMigratedException`:
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
>> >>>>> might be
>> >>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
>> >>>>> back to
>> >>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>> >> store
>> >>>>>>>>> handle.
>> >>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>> >>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>> >>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>> >>>>> `RUNNING`
>> >>>>>>>>>>>> state.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> Why do we need to distinguish between
>> >>>>>>>>> `KafkaStreamsNotRunningException`
>> >>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
>> >> instance and
>> >>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
>> >> the
>> >>>>>>>>> instance,
>> >>>>>>>>>>>>> because that is the level of granularity in which we
>> >>>>> enable/disable
>> >>>>>>>>> IQ
>> >>>>>>>>>>>> atm.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
>> >>>>> user
>> >>>>>>>>> need to
>> >>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
>> >>>>> handle is
>> >>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
>> >>>>>>>>> difference
>> >>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>> >> "blind"
>> >>>>>>> retries
>> >>>>>>>>>>>>> that either resolve (if the store is still on the same
>> >> instance
>> >>>>>>> after
>> >>>>>>>>>>>>> rebalancing finishes, or changes to
>> >>>>> `StateStoreMigratedException` if
>> >>>>>>>>> the
>> >>>>>>>>>>>>> store was migrated away during rebalancing).
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> -Matthias
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>> >>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9>
>> >> <https://shorturl.at/CDNT9>
>> >>>>> <https://shorturl.at/CDNT9>
>> >>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>>>>> <https://shorturl.at/CDNT9>
>> >>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> Please use the following instead:
>> >> https://shorturl.at/bkKQU
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> ---
>> >>>>>>>>>>>>>> Vito
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
>> >>>>> vito@is-land.com.tw>
>> >>>>>>>>> wrote:
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Thanks, Matthias!
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
>> >>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>> >>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
>> >>>>> handed out
>> >>>>>>>>> the
>> >>>>>>>>>>>>>>> store handle. We may throw
>> >> `KafkaStreamsNotRunningException`
>> >>>>> or
>> >>>>>>>>>>>>>>> `StateStoreMigratedException`.
>> >>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
>> >> not
>> >>>>>>>>> running(
>> >>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
>> >>>>> `StateStoreMigratedException`
>> >>>>>>>>> when
>> >>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
>> >> we
>> >>>>> do not
>> >>>>>>>>> need
>> >>>>>>>>>>>> to
>> >>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
>> >>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> I understand your point. I rename
>> >>>>>>>>> `StreamThreadNotRunningException` to
>> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> About check unknown state store names:
>> >>>>>>>>>>>>>>> Thank you for the hint. I add a new type
>> >>>>>>>>> `UnknownStateStoreException`
>> >>>>>>>>>>>> for
>> >>>>>>>>>>>>>>> this case.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> Also, we should still have fatal exception
>> >>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
>> >> remove
>> >>>>> it?
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Thank you point this, already add it again.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> The KIP already updated, please take a look.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> ---
>> >>>>>>>>>>>>>>> Vito
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by John Roesler <vv...@apache.org>.
Thanks for your patience, Vito!
-John

On Thu, Jan 16, 2020, at 19:12, Vito Jeng wrote:
> Thanks Bill, John and Matthias. Glad you guys joined this discussion.
> I got a lot out of the discussion.
> 
> I would like to update KIP-216 base on John's suggestion to remove the
> category.
> 
> 
> ---
> Vito
> 
> 
> On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
> > > Nevertheless, if we omit the categorization, it’s moot.
> >
> > Ack.
> >
> > I am fine to remove the middle tier. As John pointed out, it might be
> > weird to have only one concrete exception type per category. We can also
> > explain in detail how to handle each exception in their JavaDocs.
> >
> >
> > -Matthias
> >
> > On 1/16/20 6:38 AM, Bill Bejeck wrote:
> > > Vito,
> > >
> > > Thanks for the updates, the KIP LGTM.
> > >
> > > -Bill
> > >
> > > On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org>
> > wrote:
> > >
> > >> Hi Vito,
> > >>
> > >> Haha, your archive game is on point!
> > >>
> > >> What Matthias said in that email is essentially what I figured was the
> > >> rationale. It makes sense, but the point I was making is that this
> > really
> > >> doesn’t seem like a good way to structure a production app. On the other
> > >> hand, considering the exception fatal has a good chance of avoiding a
> > >> frustrating debug session if you just forgot to call start.
> > >>
> > >> Nevertheless, if we omit the categorization, it’s moot.
> > >>
> > >> It would be easy to add a categorization layer later if we want it, but
> > >> not very easy to change it if we get it wrong.
> > >>
> > >> Thanks for your consideration!
> > >> -John
> > >>
> > >> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
> > >>> Hi John,
> > >>>
> > >>> About `StreamsNotStartedException is strange` --
> > >>> The original idea came from Matthias, two years ago. :)
> > >>> You can reference here:
> > >>>
> > >>
> > https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> > >>>
> > >>> About omitting the categorization --
> > >>> It looks reasonable. I'm fine with omitting the categorization but not
> > >> very
> > >>> sure it is a good choice.
> > >>> Does any other folks provide opinion?
> > >>>
> > >>>
> > >>> Hi, folks,
> > >>>
> > >>> Just update the KIP-216, please take a look.
> > >>>
> > >>> ---
> > >>> Vito
> > >>>
> > >>>
> > >>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> > >>>
> > >>>>
> > >>>> Hi, folks,
> > >>>>
> > >>>> Thank you suggestion, really appreciate it. :)
> > >>>> I understand your concern. I'll merge StreamsNotRunningException and
> > >>>> StateStoreNotAvailableException.
> > >>>>
> > >>>>
> > >>>> ---
> > >>>> Vito
> > >>>>
> > >>>>
> > >>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org>
> > >> wrote:
> > >>>>
> > >>>>> Hey Vito,
> > >>>>>
> > >>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
> > >>>>> Hopefully, we can close it out soon.
> > >>>>>
> > >>>>> I agree we can drop StreamsNotRunningException in favor of
> > >>>>> just StateStoreNotAvailableException.
> > >>>>>
> > >>>>> Unfortunately, I have some higher-level concerns. The value
> > >>>>> of these exceptions is that they tell you how to handle the
> > >>>>> various situations that can arise while querying a distributed
> > >>>>> data store.
> > >>>>>
> > >>>>> Ideally, as a caller, I should be able to just catch "retriable" or
> > >>>>> "fatal" and handle them appropriately. Otherwise, there's no
> > >>>>> point in having categories, and we should just have all the
> > >>>>> exceptions extend InvalidStateStoreException.
> > >>>>>
> > >>>>> Presently, it's not possible to tell from just the
> > >>>>> "retriable"/"fatal" distinction what to do. You  can tell
> > >>>>> from the descriptions of the various exceptions. E.g.:
> > >>>>>
> > >>>>> Retriable:
> > >>>>>  * StreamsRebalancingException: the exact same call
> > >>>>>     should just be retried until the rebalance is complete
> > >>>>>  * StateStoreMigratedException: the store handle is
> > >>>>>     now invalid, so you need to re-discover the instance
> > >>>>>     and get a new handle on that instance. In other words,
> > >>>>>     the query itself may be valid, but the particular method
> > >>>>>     invocation on this particular instance has encountered
> > >>>>>     a fatal exception.
> > >>>>>
> > >>>>> Fatal:
> > >>>>>  * UnknownStateStoreException: this is truly fatal. No amount
> > >>>>>     of retrying or re-discovering is going to get you a handle on a
> > >>>>>     store that doesn't exist in the cluster.
> > >>>>>  * StateStoreNotAvailableException: this is actually recoverable,
> > >>>>>     since the store might exist in the cluster, but isn't available
> > on
> > >>>>>     this particular instance (which is shut down or whatever).
> > >>>>>
> > >>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> > >>>>> with omitting the categorization and just having 5 subclasses
> > >>>>> of InvalidStateStoreException. Each of them would tell you
> > >>>>> how to handle them, and it's not too many to really
> > >>>>> understand and handle each one.
> > >>>>>
> > >>>>> If you really want to have a middle tier, I'd recommend:
> > >>>>> * RetryableStateStoreException: the exact same call
> > >>>>>     should be repeated.
> > >>>>> * RecoverableStateStoreException: the store handle
> > >>>>>     should be discarded and the caller should re-discover
> > >>>>>     the location of the store and repeat the query on the
> > >>>>>     correct instance.
> > >>>>> * FatalStateStoreException: the query/request is totally
> > >>>>>     invalid and will never succeed.
> > >>>>>
> > >>>>> However, attempting to categorize the proposed exceptions
> > >>>>> reveals even problems with this categorization:
> > >>>>> Retriable:
> > >>>>> * StreamsRebalancingException
> > >>>>> Recoverable:
> > >>>>> * StateStoreMigratedException
> > >>>>> * StreamsNotRunningException
> > >>>>> Fatal:
> > >>>>> * UnknownStateStoreException
> > >>>>>
> > >>>>> But StreamsNotStartedException is strange... It means that
> > >>>>> one code path got a handle on a specific KafkaStreams object
> > >>>>> instance and sent it a query before another code path
> > >>>>> invoked the start() method on the exact same object instance.
> > >>>>> It seems like the most likely scenario is that whoever wrote
> > >>>>> the program just forgot to call start() before querying, in
> > >>>>> which case, retrying isn't going to help, and a fatal exception
> > >>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
> > >>>>> experience" problem, and making it fatal would be more
> > >>>>> helpful. Even in a production context, there's no reason not
> > >>>>> to sequence your application startup such that you don't
> > >>>>> accept queries until after Streams is started. Thus, I guess
> > >>>>> I'd categorize it under "fatal".
> > >>>>>
> > >>>>> Regardless of whether you make it fatal or retriable, you'd
> > >>>>> still have a whole category with only one exception in it,
> > >>>>> and the other two categories only have two exceptions.
> > >>>>> Plus, as you pointed out in the KIP, you can't get all
> > >>>>> exceptions in all cases anyway:
> > >>>>> * store() can only throw NotStarted, NotRunning,
> > >>>>>     and Unknown
> > >>>>> * actual store queries can only throw Rebalancing,
> > >>>>>     Migrated, and NotRunning
> > >>>>>
> > >>>>> Thus, in practice also, there are exactly three categories
> > >>>>> and also exactly three exception types. It doesn't seem
> > >>>>> like there's a great advantage to the categories here. To
> > >>>>> avoid the categorization problem and also to clarify what
> > >>>>> exceptions can actually be thrown in different circumstances,
> > >>>>> it seems like we should just:
> > >>>>> * get rid of the middle tier and make all the exceptions
> > >>>>>     extend InvalidStateStoreException
> > >>>>> * drop StateStoreNotAvailableException in favor of
> > >>>>>     StreamsNotRunningException
> > >>>>> * clearly document on all public methods which exceptions
> > >>>>>     need to be handled
> > >>>>>
> > >>>>> How do you feel about this?
> > >>>>> Thanks,
> > >>>>> -John
> > >>>>>
> > >>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> > >>>>>> Thanks for KIP Vito.
> > >>>>>>
> > >>>>>> Overall the KIP LGTM, but I'd have to agree with others on merging
> > >> the
> > >>>>>> `StreamsNotRunningException` and `StateStoreNotAvailableException`
> > >>>>> classes.
> > >>>>>>
> > >>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> > >>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
> > >> when to
> > >>>>>> use the different
> > >>>>>> exceptions.  Maybe a good middle ground would be to have a detailed
> > >>>>>> exception message.
> > >>>>>>
> > >>>>>> The KIP freeze is close, so I think if we can agree on this, we can
> > >>>>> wrap up
> > >>>>>> the voting soon.
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>> Bill
> > >>>>>>
> > >>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
> > >> matthias@confluent.io>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Vito,
> > >>>>>>>
> > >>>>>>> It's still unclear to me what the advantage is, to have both
> > >>>>>>> `StreamsNotRunningException` and
> > >> `StateStoreNotAvailableException`?
> > >>>>>>>
> > >>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
> > >> ERROR`
> > >>>>>>> and thus, for a user point of view, why does it matter if the
> > >> store is
> > >>>>>>> closed on not? I don't understand why/how this information would
> > >> be
> > >>>>>>> useful? Do you have a concrete example in mind how a user would
> > >> react
> > >>>>>>> differently to both exceptions?
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
> > >> best
> > >>>>> to
> > >>>>>>> actually do this on a per-query basis, ie, have an overload
> > >>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow to
> > >>>>>>> _disable_ the exception and opt-in to query a active store during
> > >>>>>>> recovery. However, as KIP-535 actually introduces this change in
> > >>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
> > >> should be
> > >>>>>>> updated. I'll follow up on the other KIP thread to raise this
> > >> point.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> -Matthias
> > >>>>>>>
> > >>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
> > >>>>>>>> Hi, Matthias & Vinoth,
> > >>>>>>>>
> > >>>>>>>> Thanks for the feedback.
> > >>>>>>>>
> > >>>>>>>>> What is still unclear to me is, what we gain by having both
> > >>>>>>>>> `StreamsNotRunningException` and
> > >>>>> `StateStoreNotAvailableException`. Both
> > >>>>>>>>> exception are thrown when KafkaStreams is in state
> > >>>>> PENDING_SHUTDOWN /
> > >>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> > >> if the
> > >>>>>>>>> state store is closed on not -- I can't query it anyway? Maybe
> > >> I
> > >>>>> miss
> > >>>>>>>>> something thought?
> > >>>>>>>>
> > >>>>>>>> Yes, both `StreamsNotRunningException` and
> > >>>>>>>> `StateStoreNotAvailableException` are fatal exception.
> > >>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
> > >> state
> > >>>>>>> store
> > >>>>>>>> related.
> > >>>>>>>> I think it would be helpful that if user need to distinguish
> > >> these
> > >>>>> two
> > >>>>>>>> different case to handle it.
> > >>>>>>>>
> > >>>>>>>> I'm not very sure, does that make sense?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> ---
> > >>>>>>>> Vito
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
> > >> vinoth@apache.org>
> > >>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> +1 on merging `StreamsNotRunningException` and
> > >>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
> > >>>>> anyway. IMO
> > >>>>>>>>> its best to have these exceptions be about the state store
> > >> (and not
> > >>>>>>> streams
> > >>>>>>>>> state), to easier understanding.
> > >>>>>>>>>
> > >>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
> > >>>>> rebalancing
> > >>>>>>>>> state. So do we need the StreamsRebalancingException?
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
> > >> matthias@confluent.io>
> > >>>>>>> wrote:
> > >>>>>>>>>> Sorry that I dropped the ball on this...
> > >>>>>>>>>>
> > >>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
> > >> start
> > >>>>> a
> > >>>>>>> VOTE
> > >>>>>>>>>> thread.
> > >>>>>>>>>>
> > >>>>>>>>>> What is still unclear to me is, what we gain by having both
> > >>>>>>>>>> `StreamsNotRunningException` and
> > >>>>> `StateStoreNotAvailableException`.
> > >>>>>>> Both
> > >>>>>>>>>> exception are thrown when KafkaStreams is in state
> > >>>>> PENDING_SHUTDOWN /
> > >>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> > >> if
> > >>>>> the
> > >>>>>>>>>> state store is closed on not -- I can't query it anyway?
> > >> Maybe I
> > >>>>> miss
> > >>>>>>>>>> something thought?
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> -Matthias
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> > >>>>>>>>>>> Sorry for the late reply, thanks for the review.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>> About `StateStoreMigratedException`:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
> > >>>>> might be
> > >>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
> > >>>>> back to
> > >>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> > >> store
> > >>>>>>> handle.
> > >>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> > >>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> > >>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> > >>>>> `RUNNING`
> > >>>>>>>>> state.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thank you point this, already updated.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Why do we need to distinguish between
> > >>>>>>> `KafkaStreamsNotRunningException`
> > >>>>>>>>>>>> and `StateStoreNotAvailableException`?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
> > >>>>> reasons, I
> > >>>>>>>>> think
> > >>>>>>>>>>> it would be helpful that the
> > >>>>>>>>>>> user can distinguish whether it is caused by the state store
> > >>>>> closed.
> > >>>>>>>>>>> (Maybe I am wrong...)
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
> > >> and
> > >>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
> > >> the
> > >>>>>>>>> instance,
> > >>>>>>>>>>>> because that is the level of granularity in which we
> > >>>>> enable/disable
> > >>>>>>>>> IQ atm.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Totally agree. Do you mean the naming of state store
> > >> exceptions?
> > >>>>>>>>>>> I don't have special reason to distinguish these two.
> > >>>>>>>>>>> Your suggestion look more reasonable for the exception
> > >> naming.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
> > >> user
> > >>>>> need
> > >>>>>>>>> to
> > >>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
> > >>>>> handle is
> > >>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
> > >>>>>>> difference
> > >>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> > >> "blind"
> > >>>>> retries
> > >>>>>>>>>>>> that either resolve (if the store is still on the same
> > >> instance
> > >>>>> after
> > >>>>>>>>>>>> rebalancing finishes, or changes to
> > >>>>> `StateStoreMigratedException` if
> > >>>>>>>>> the
> > >>>>>>>>>>>> store was migrated away during rebalancing).
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Nice, it's great! Thank you.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> The KIP already updated, please take a look. :)
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> > >>>>>>> matthias@confluent.io
> > >>>>>>>>>>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Any update on this KIP?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> > >>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
> > >> busy.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> About `StateStoreMigratedException`:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
> > >>>>> might be
> > >>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
> > >>>>> back to
> > >>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> > >> store
> > >>>>>>>>> handle.
> > >>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> > >>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> > >>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> > >>>>> `RUNNING`
> > >>>>>>>>>>>> state.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Why do we need to distinguish between
> > >>>>>>>>> `KafkaStreamsNotRunningException`
> > >>>>>>>>>>>>> and `StateStoreNotAvailableException`?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
> > >> instance and
> > >>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
> > >> the
> > >>>>>>>>> instance,
> > >>>>>>>>>>>>> because that is the level of granularity in which we
> > >>>>> enable/disable
> > >>>>>>>>> IQ
> > >>>>>>>>>>>> atm.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
> > >>>>> user
> > >>>>>>>>> need to
> > >>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
> > >>>>> handle is
> > >>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
> > >>>>>>>>> difference
> > >>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> > >> "blind"
> > >>>>>>> retries
> > >>>>>>>>>>>>> that either resolve (if the store is still on the same
> > >> instance
> > >>>>>>> after
> > >>>>>>>>>>>>> rebalancing finishes, or changes to
> > >>>>> `StateStoreMigratedException` if
> > >>>>>>>>> the
> > >>>>>>>>>>>>> store was migrated away during rebalancing).
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> > >>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> > <https://shorturl.at/CDNT9>
> > >> <https://shorturl.at/CDNT9>
> > >>>>> <https://shorturl.at/CDNT9>
> > >>>>>>> <https://shorturl.at/CDNT9>
> > >>>>>>>>> <https://shorturl.at/CDNT9>
> > >>>>>>>>>>>> <https://shorturl.at/CDNT9>
> > >>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Please use the following instead:
> > >> https://shorturl.at/bkKQU
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> ---
> > >>>>>>>>>>>>>> Vito
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
> > >>>>> vito@is-land.com.tw>
> > >>>>>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks, Matthias!
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
> > >>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> > >>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
> > >>>>> handed out
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> store handle. We may throw
> > >> `KafkaStreamsNotRunningException`
> > >>>>> or
> > >>>>>>>>>>>>>>> `StateStoreMigratedException`.
> > >>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> > >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
> > >> not
> > >>>>>>>>> running(
> > >>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
> > >>>>> `StateStoreMigratedException`
> > >>>>>>>>> when
> > >>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
> > >> we
> > >>>>> do not
> > >>>>>>>>> need
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
> > >>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I understand your point. I rename
> > >>>>>>>>> `StreamThreadNotRunningException` to
> > >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> About check unknown state store names:
> > >>>>>>>>>>>>>>> Thank you for the hint. I add a new type
> > >>>>>>>>> `UnknownStateStoreException`
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>> this case.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Also, we should still have fatal exception
> > >>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
> > >> remove
> > >>>>> it?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thank you point this, already add it again.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> The KIP already updated, please take a look.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> ---
> > >>>>>>>>>>>>>>> Vito
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
> >
> >
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Thanks Bill, John and Matthias. Glad you guys joined this discussion.
I got a lot out of the discussion.

I would like to update KIP-216 base on John's suggestion to remove the
category.


---
Vito


On Fri, Jan 17, 2020 at 2:30 AM Matthias J. Sax <ma...@confluent.io>
wrote:

> > Nevertheless, if we omit the categorization, it’s moot.
>
> Ack.
>
> I am fine to remove the middle tier. As John pointed out, it might be
> weird to have only one concrete exception type per category. We can also
> explain in detail how to handle each exception in their JavaDocs.
>
>
> -Matthias
>
> On 1/16/20 6:38 AM, Bill Bejeck wrote:
> > Vito,
> >
> > Thanks for the updates, the KIP LGTM.
> >
> > -Bill
> >
> > On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org>
> wrote:
> >
> >> Hi Vito,
> >>
> >> Haha, your archive game is on point!
> >>
> >> What Matthias said in that email is essentially what I figured was the
> >> rationale. It makes sense, but the point I was making is that this
> really
> >> doesn’t seem like a good way to structure a production app. On the other
> >> hand, considering the exception fatal has a good chance of avoiding a
> >> frustrating debug session if you just forgot to call start.
> >>
> >> Nevertheless, if we omit the categorization, it’s moot.
> >>
> >> It would be easy to add a categorization layer later if we want it, but
> >> not very easy to change it if we get it wrong.
> >>
> >> Thanks for your consideration!
> >> -John
> >>
> >> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
> >>> Hi John,
> >>>
> >>> About `StreamsNotStartedException is strange` --
> >>> The original idea came from Matthias, two years ago. :)
> >>> You can reference here:
> >>>
> >>
> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> >>>
> >>> About omitting the categorization --
> >>> It looks reasonable. I'm fine with omitting the categorization but not
> >> very
> >>> sure it is a good choice.
> >>> Does any other folks provide opinion?
> >>>
> >>>
> >>> Hi, folks,
> >>>
> >>> Just update the KIP-216, please take a look.
> >>>
> >>> ---
> >>> Vito
> >>>
> >>>
> >>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> >>>
> >>>>
> >>>> Hi, folks,
> >>>>
> >>>> Thank you suggestion, really appreciate it. :)
> >>>> I understand your concern. I'll merge StreamsNotRunningException and
> >>>> StateStoreNotAvailableException.
> >>>>
> >>>>
> >>>> ---
> >>>> Vito
> >>>>
> >>>>
> >>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org>
> >> wrote:
> >>>>
> >>>>> Hey Vito,
> >>>>>
> >>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
> >>>>> Hopefully, we can close it out soon.
> >>>>>
> >>>>> I agree we can drop StreamsNotRunningException in favor of
> >>>>> just StateStoreNotAvailableException.
> >>>>>
> >>>>> Unfortunately, I have some higher-level concerns. The value
> >>>>> of these exceptions is that they tell you how to handle the
> >>>>> various situations that can arise while querying a distributed
> >>>>> data store.
> >>>>>
> >>>>> Ideally, as a caller, I should be able to just catch "retriable" or
> >>>>> "fatal" and handle them appropriately. Otherwise, there's no
> >>>>> point in having categories, and we should just have all the
> >>>>> exceptions extend InvalidStateStoreException.
> >>>>>
> >>>>> Presently, it's not possible to tell from just the
> >>>>> "retriable"/"fatal" distinction what to do. You  can tell
> >>>>> from the descriptions of the various exceptions. E.g.:
> >>>>>
> >>>>> Retriable:
> >>>>>  * StreamsRebalancingException: the exact same call
> >>>>>     should just be retried until the rebalance is complete
> >>>>>  * StateStoreMigratedException: the store handle is
> >>>>>     now invalid, so you need to re-discover the instance
> >>>>>     and get a new handle on that instance. In other words,
> >>>>>     the query itself may be valid, but the particular method
> >>>>>     invocation on this particular instance has encountered
> >>>>>     a fatal exception.
> >>>>>
> >>>>> Fatal:
> >>>>>  * UnknownStateStoreException: this is truly fatal. No amount
> >>>>>     of retrying or re-discovering is going to get you a handle on a
> >>>>>     store that doesn't exist in the cluster.
> >>>>>  * StateStoreNotAvailableException: this is actually recoverable,
> >>>>>     since the store might exist in the cluster, but isn't available
> on
> >>>>>     this particular instance (which is shut down or whatever).
> >>>>>
> >>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> >>>>> with omitting the categorization and just having 5 subclasses
> >>>>> of InvalidStateStoreException. Each of them would tell you
> >>>>> how to handle them, and it's not too many to really
> >>>>> understand and handle each one.
> >>>>>
> >>>>> If you really want to have a middle tier, I'd recommend:
> >>>>> * RetryableStateStoreException: the exact same call
> >>>>>     should be repeated.
> >>>>> * RecoverableStateStoreException: the store handle
> >>>>>     should be discarded and the caller should re-discover
> >>>>>     the location of the store and repeat the query on the
> >>>>>     correct instance.
> >>>>> * FatalStateStoreException: the query/request is totally
> >>>>>     invalid and will never succeed.
> >>>>>
> >>>>> However, attempting to categorize the proposed exceptions
> >>>>> reveals even problems with this categorization:
> >>>>> Retriable:
> >>>>> * StreamsRebalancingException
> >>>>> Recoverable:
> >>>>> * StateStoreMigratedException
> >>>>> * StreamsNotRunningException
> >>>>> Fatal:
> >>>>> * UnknownStateStoreException
> >>>>>
> >>>>> But StreamsNotStartedException is strange... It means that
> >>>>> one code path got a handle on a specific KafkaStreams object
> >>>>> instance and sent it a query before another code path
> >>>>> invoked the start() method on the exact same object instance.
> >>>>> It seems like the most likely scenario is that whoever wrote
> >>>>> the program just forgot to call start() before querying, in
> >>>>> which case, retrying isn't going to help, and a fatal exception
> >>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
> >>>>> experience" problem, and making it fatal would be more
> >>>>> helpful. Even in a production context, there's no reason not
> >>>>> to sequence your application startup such that you don't
> >>>>> accept queries until after Streams is started. Thus, I guess
> >>>>> I'd categorize it under "fatal".
> >>>>>
> >>>>> Regardless of whether you make it fatal or retriable, you'd
> >>>>> still have a whole category with only one exception in it,
> >>>>> and the other two categories only have two exceptions.
> >>>>> Plus, as you pointed out in the KIP, you can't get all
> >>>>> exceptions in all cases anyway:
> >>>>> * store() can only throw NotStarted, NotRunning,
> >>>>>     and Unknown
> >>>>> * actual store queries can only throw Rebalancing,
> >>>>>     Migrated, and NotRunning
> >>>>>
> >>>>> Thus, in practice also, there are exactly three categories
> >>>>> and also exactly three exception types. It doesn't seem
> >>>>> like there's a great advantage to the categories here. To
> >>>>> avoid the categorization problem and also to clarify what
> >>>>> exceptions can actually be thrown in different circumstances,
> >>>>> it seems like we should just:
> >>>>> * get rid of the middle tier and make all the exceptions
> >>>>>     extend InvalidStateStoreException
> >>>>> * drop StateStoreNotAvailableException in favor of
> >>>>>     StreamsNotRunningException
> >>>>> * clearly document on all public methods which exceptions
> >>>>>     need to be handled
> >>>>>
> >>>>> How do you feel about this?
> >>>>> Thanks,
> >>>>> -John
> >>>>>
> >>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> >>>>>> Thanks for KIP Vito.
> >>>>>>
> >>>>>> Overall the KIP LGTM, but I'd have to agree with others on merging
> >> the
> >>>>>> `StreamsNotRunningException` and `StateStoreNotAvailableException`
> >>>>> classes.
> >>>>>>
> >>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> >>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
> >> when to
> >>>>>> use the different
> >>>>>> exceptions.  Maybe a good middle ground would be to have a detailed
> >>>>>> exception message.
> >>>>>>
> >>>>>> The KIP freeze is close, so I think if we can agree on this, we can
> >>>>> wrap up
> >>>>>> the voting soon.
> >>>>>>
> >>>>>> Thanks,
> >>>>>> Bill
> >>>>>>
> >>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
> >> matthias@confluent.io>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Vito,
> >>>>>>>
> >>>>>>> It's still unclear to me what the advantage is, to have both
> >>>>>>> `StreamsNotRunningException` and
> >> `StateStoreNotAvailableException`?
> >>>>>>>
> >>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
> >> ERROR`
> >>>>>>> and thus, for a user point of view, why does it matter if the
> >> store is
> >>>>>>> closed on not? I don't understand why/how this information would
> >> be
> >>>>>>> useful? Do you have a concrete example in mind how a user would
> >> react
> >>>>>>> differently to both exceptions?
> >>>>>>>
> >>>>>>>
> >>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
> >> best
> >>>>> to
> >>>>>>> actually do this on a per-query basis, ie, have an overload
> >>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow to
> >>>>>>> _disable_ the exception and opt-in to query a active store during
> >>>>>>> recovery. However, as KIP-535 actually introduces this change in
> >>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
> >> should be
> >>>>>>> updated. I'll follow up on the other KIP thread to raise this
> >> point.
> >>>>>>>
> >>>>>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
> >>>>>>>> Hi, Matthias & Vinoth,
> >>>>>>>>
> >>>>>>>> Thanks for the feedback.
> >>>>>>>>
> >>>>>>>>> What is still unclear to me is, what we gain by having both
> >>>>>>>>> `StreamsNotRunningException` and
> >>>>> `StateStoreNotAvailableException`. Both
> >>>>>>>>> exception are thrown when KafkaStreams is in state
> >>>>> PENDING_SHUTDOWN /
> >>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> >> if the
> >>>>>>>>> state store is closed on not -- I can't query it anyway? Maybe
> >> I
> >>>>> miss
> >>>>>>>>> something thought?
> >>>>>>>>
> >>>>>>>> Yes, both `StreamsNotRunningException` and
> >>>>>>>> `StateStoreNotAvailableException` are fatal exception.
> >>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
> >> state
> >>>>>>> store
> >>>>>>>> related.
> >>>>>>>> I think it would be helpful that if user need to distinguish
> >> these
> >>>>> two
> >>>>>>>> different case to handle it.
> >>>>>>>>
> >>>>>>>> I'm not very sure, does that make sense?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> ---
> >>>>>>>> Vito
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
> >> vinoth@apache.org>
> >>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> +1 on merging `StreamsNotRunningException` and
> >>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
> >>>>> anyway. IMO
> >>>>>>>>> its best to have these exceptions be about the state store
> >> (and not
> >>>>>>> streams
> >>>>>>>>> state), to easier understanding.
> >>>>>>>>>
> >>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
> >>>>> rebalancing
> >>>>>>>>> state. So do we need the StreamsRebalancingException?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
> >> matthias@confluent.io>
> >>>>>>> wrote:
> >>>>>>>>>> Sorry that I dropped the ball on this...
> >>>>>>>>>>
> >>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
> >> start
> >>>>> a
> >>>>>>> VOTE
> >>>>>>>>>> thread.
> >>>>>>>>>>
> >>>>>>>>>> What is still unclear to me is, what we gain by having both
> >>>>>>>>>> `StreamsNotRunningException` and
> >>>>> `StateStoreNotAvailableException`.
> >>>>>>> Both
> >>>>>>>>>> exception are thrown when KafkaStreams is in state
> >>>>> PENDING_SHUTDOWN /
> >>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> >> if
> >>>>> the
> >>>>>>>>>> state store is closed on not -- I can't query it anyway?
> >> Maybe I
> >>>>> miss
> >>>>>>>>>> something thought?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> >>>>>>>>>>> Sorry for the late reply, thanks for the review.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>> About `StateStoreMigratedException`:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
> >>>>> might be
> >>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
> >>>>> back to
> >>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> >> store
> >>>>>>> handle.
> >>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> >>>>> `RUNNING`
> >>>>>>>>> state.
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Thank you point this, already updated.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Why do we need to distinguish between
> >>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
> >>>>> reasons, I
> >>>>>>>>> think
> >>>>>>>>>>> it would be helpful that the
> >>>>>>>>>>> user can distinguish whether it is caused by the state store
> >>>>> closed.
> >>>>>>>>>>> (Maybe I am wrong...)
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
> >> and
> >>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
> >> the
> >>>>>>>>> instance,
> >>>>>>>>>>>> because that is the level of granularity in which we
> >>>>> enable/disable
> >>>>>>>>> IQ atm.
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Totally agree. Do you mean the naming of state store
> >> exceptions?
> >>>>>>>>>>> I don't have special reason to distinguish these two.
> >>>>>>>>>>> Your suggestion look more reasonable for the exception
> >> naming.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
> >> user
> >>>>> need
> >>>>>>>>> to
> >>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
> >>>>> handle is
> >>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
> >>>>>>> difference
> >>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >> "blind"
> >>>>> retries
> >>>>>>>>>>>> that either resolve (if the store is still on the same
> >> instance
> >>>>> after
> >>>>>>>>>>>> rebalancing finishes, or changes to
> >>>>> `StateStoreMigratedException` if
> >>>>>>>>> the
> >>>>>>>>>>>> store was migrated away during rebalancing).
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Nice, it's great! Thank you.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> The KIP already updated, please take a look. :)
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> >>>>>>> matthias@confluent.io
> >>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Any update on this KIP?
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
> >> busy.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> About `StateStoreMigratedException`:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
> >>>>> might be
> >>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
> >>>>> back to
> >>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
> >> store
> >>>>>>>>> handle.
> >>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
> >>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
> >>>>> `RUNNING`
> >>>>>>>>>>>> state.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Why do we need to distinguish between
> >>>>>>>>> `KafkaStreamsNotRunningException`
> >>>>>>>>>>>>> and `StateStoreNotAvailableException`?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
> >> instance and
> >>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
> >> the
> >>>>>>>>> instance,
> >>>>>>>>>>>>> because that is the level of granularity in which we
> >>>>> enable/disable
> >>>>>>>>> IQ
> >>>>>>>>>>>> atm.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
> >>>>> user
> >>>>>>>>> need to
> >>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
> >>>>> handle is
> >>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
> >>>>>>>>> difference
> >>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
> >> "blind"
> >>>>>>> retries
> >>>>>>>>>>>>> that either resolve (if the store is still on the same
> >> instance
> >>>>>>> after
> >>>>>>>>>>>>> rebalancing finishes, or changes to
> >>>>> `StateStoreMigratedException` if
> >>>>>>>>> the
> >>>>>>>>>>>>> store was migrated away during rebalancing).
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >> <https://shorturl.at/CDNT9>
> >>>>> <https://shorturl.at/CDNT9>
> >>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>> <https://shorturl.at/CDNT9>
> >>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Please use the following instead:
> >> https://shorturl.at/bkKQU
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
> >>>>> vito@is-land.com.tw>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks, Matthias!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
> >>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
> >>>>> handed out
> >>>>>>>>> the
> >>>>>>>>>>>>>>> store handle. We may throw
> >> `KafkaStreamsNotRunningException`
> >>>>> or
> >>>>>>>>>>>>>>> `StateStoreMigratedException`.
> >>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
> >> not
> >>>>>>>>> running(
> >>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
> >>>>> `StateStoreMigratedException`
> >>>>>>>>> when
> >>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
> >> we
> >>>>> do not
> >>>>>>>>> need
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
> >>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I understand your point. I rename
> >>>>>>>>> `StreamThreadNotRunningException` to
> >>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> About check unknown state store names:
> >>>>>>>>>>>>>>> Thank you for the hint. I add a new type
> >>>>>>>>> `UnknownStateStoreException`
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>> this case.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Also, we should still have fatal exception
> >>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
> >> remove
> >>>>> it?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thank you point this, already add it again.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The KIP already updated, please take a look.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> ---
> >>>>>>>>>>>>>>> Vito
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
> Nevertheless, if we omit the categorization, it’s moot.

Ack.

I am fine to remove the middle tier. As John pointed out, it might be
weird to have only one concrete exception type per category. We can also
explain in detail how to handle each exception in their JavaDocs.


-Matthias

On 1/16/20 6:38 AM, Bill Bejeck wrote:
> Vito,
> 
> Thanks for the updates, the KIP LGTM.
> 
> -Bill
> 
> On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org> wrote:
> 
>> Hi Vito,
>>
>> Haha, your archive game is on point!
>>
>> What Matthias said in that email is essentially what I figured was the
>> rationale. It makes sense, but the point I was making is that this really
>> doesn’t seem like a good way to structure a production app. On the other
>> hand, considering the exception fatal has a good chance of avoiding a
>> frustrating debug session if you just forgot to call start.
>>
>> Nevertheless, if we omit the categorization, it’s moot.
>>
>> It would be easy to add a categorization layer later if we want it, but
>> not very easy to change it if we get it wrong.
>>
>> Thanks for your consideration!
>> -John
>>
>> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
>>> Hi John,
>>>
>>> About `StreamsNotStartedException is strange` --
>>> The original idea came from Matthias, two years ago. :)
>>> You can reference here:
>>>
>> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
>>>
>>> About omitting the categorization --
>>> It looks reasonable. I'm fine with omitting the categorization but not
>> very
>>> sure it is a good choice.
>>> Does any other folks provide opinion?
>>>
>>>
>>> Hi, folks,
>>>
>>> Just update the KIP-216, please take a look.
>>>
>>> ---
>>> Vito
>>>
>>>
>>> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw> wrote:
>>>
>>>>
>>>> Hi, folks,
>>>>
>>>> Thank you suggestion, really appreciate it. :)
>>>> I understand your concern. I'll merge StreamsNotRunningException and
>>>> StateStoreNotAvailableException.
>>>>
>>>>
>>>> ---
>>>> Vito
>>>>
>>>>
>>>> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org>
>> wrote:
>>>>
>>>>> Hey Vito,
>>>>>
>>>>> Yes, thanks for the KIP. Sorry the discussion has been so long.
>>>>> Hopefully, we can close it out soon.
>>>>>
>>>>> I agree we can drop StreamsNotRunningException in favor of
>>>>> just StateStoreNotAvailableException.
>>>>>
>>>>> Unfortunately, I have some higher-level concerns. The value
>>>>> of these exceptions is that they tell you how to handle the
>>>>> various situations that can arise while querying a distributed
>>>>> data store.
>>>>>
>>>>> Ideally, as a caller, I should be able to just catch "retriable" or
>>>>> "fatal" and handle them appropriately. Otherwise, there's no
>>>>> point in having categories, and we should just have all the
>>>>> exceptions extend InvalidStateStoreException.
>>>>>
>>>>> Presently, it's not possible to tell from just the
>>>>> "retriable"/"fatal" distinction what to do. You  can tell
>>>>> from the descriptions of the various exceptions. E.g.:
>>>>>
>>>>> Retriable:
>>>>>  * StreamsRebalancingException: the exact same call
>>>>>     should just be retried until the rebalance is complete
>>>>>  * StateStoreMigratedException: the store handle is
>>>>>     now invalid, so you need to re-discover the instance
>>>>>     and get a new handle on that instance. In other words,
>>>>>     the query itself may be valid, but the particular method
>>>>>     invocation on this particular instance has encountered
>>>>>     a fatal exception.
>>>>>
>>>>> Fatal:
>>>>>  * UnknownStateStoreException: this is truly fatal. No amount
>>>>>     of retrying or re-discovering is going to get you a handle on a
>>>>>     store that doesn't exist in the cluster.
>>>>>  * StateStoreNotAvailableException: this is actually recoverable,
>>>>>     since the store might exist in the cluster, but isn't available on
>>>>>     this particular instance (which is shut down or whatever).
>>>>>
>>>>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
>>>>> with omitting the categorization and just having 5 subclasses
>>>>> of InvalidStateStoreException. Each of them would tell you
>>>>> how to handle them, and it's not too many to really
>>>>> understand and handle each one.
>>>>>
>>>>> If you really want to have a middle tier, I'd recommend:
>>>>> * RetryableStateStoreException: the exact same call
>>>>>     should be repeated.
>>>>> * RecoverableStateStoreException: the store handle
>>>>>     should be discarded and the caller should re-discover
>>>>>     the location of the store and repeat the query on the
>>>>>     correct instance.
>>>>> * FatalStateStoreException: the query/request is totally
>>>>>     invalid and will never succeed.
>>>>>
>>>>> However, attempting to categorize the proposed exceptions
>>>>> reveals even problems with this categorization:
>>>>> Retriable:
>>>>> * StreamsRebalancingException
>>>>> Recoverable:
>>>>> * StateStoreMigratedException
>>>>> * StreamsNotRunningException
>>>>> Fatal:
>>>>> * UnknownStateStoreException
>>>>>
>>>>> But StreamsNotStartedException is strange... It means that
>>>>> one code path got a handle on a specific KafkaStreams object
>>>>> instance and sent it a query before another code path
>>>>> invoked the start() method on the exact same object instance.
>>>>> It seems like the most likely scenario is that whoever wrote
>>>>> the program just forgot to call start() before querying, in
>>>>> which case, retrying isn't going to help, and a fatal exception
>>>>> is more appropriate. I.e., it sounds like a "first 15 minutes
>>>>> experience" problem, and making it fatal would be more
>>>>> helpful. Even in a production context, there's no reason not
>>>>> to sequence your application startup such that you don't
>>>>> accept queries until after Streams is started. Thus, I guess
>>>>> I'd categorize it under "fatal".
>>>>>
>>>>> Regardless of whether you make it fatal or retriable, you'd
>>>>> still have a whole category with only one exception in it,
>>>>> and the other two categories only have two exceptions.
>>>>> Plus, as you pointed out in the KIP, you can't get all
>>>>> exceptions in all cases anyway:
>>>>> * store() can only throw NotStarted, NotRunning,
>>>>>     and Unknown
>>>>> * actual store queries can only throw Rebalancing,
>>>>>     Migrated, and NotRunning
>>>>>
>>>>> Thus, in practice also, there are exactly three categories
>>>>> and also exactly three exception types. It doesn't seem
>>>>> like there's a great advantage to the categories here. To
>>>>> avoid the categorization problem and also to clarify what
>>>>> exceptions can actually be thrown in different circumstances,
>>>>> it seems like we should just:
>>>>> * get rid of the middle tier and make all the exceptions
>>>>>     extend InvalidStateStoreException
>>>>> * drop StateStoreNotAvailableException in favor of
>>>>>     StreamsNotRunningException
>>>>> * clearly document on all public methods which exceptions
>>>>>     need to be handled
>>>>>
>>>>> How do you feel about this?
>>>>> Thanks,
>>>>> -John
>>>>>
>>>>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
>>>>>> Thanks for KIP Vito.
>>>>>>
>>>>>> Overall the KIP LGTM, but I'd have to agree with others on merging
>> the
>>>>>> `StreamsNotRunningException` and `StateStoreNotAvailableException`
>>>>> classes.
>>>>>>
>>>>>> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
>>>>>> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
>> when to
>>>>>> use the different
>>>>>> exceptions.  Maybe a good middle ground would be to have a detailed
>>>>>> exception message.
>>>>>>
>>>>>> The KIP freeze is close, so I think if we can agree on this, we can
>>>>> wrap up
>>>>>> the voting soon.
>>>>>>
>>>>>> Thanks,
>>>>>> Bill
>>>>>>
>>>>>> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
>> matthias@confluent.io>
>>>>>> wrote:
>>>>>>
>>>>>>> Vito,
>>>>>>>
>>>>>>> It's still unclear to me what the advantage is, to have both
>>>>>>> `StreamsNotRunningException` and
>> `StateStoreNotAvailableException`?
>>>>>>>
>>>>>>> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
>> ERROR`
>>>>>>> and thus, for a user point of view, why does it matter if the
>> store is
>>>>>>> closed on not? I don't understand why/how this information would
>> be
>>>>>>> useful? Do you have a concrete example in mind how a user would
>> react
>>>>>>> differently to both exceptions?
>>>>>>>
>>>>>>>
>>>>>>> @Vinoth: about `StreamsRebalancingException` -- to me, it seems
>> best
>>>>> to
>>>>>>> actually do this on a per-query basis, ie, have an overload
>>>>>>> `KafkaStreams#store(...)` that takes a boolean flag that allow to
>>>>>>> _disable_ the exception and opt-in to query a active store during
>>>>>>> recovery. However, as KIP-535 actually introduces this change in
>>>>>>> behavior, I think KIP-216 should not cover this, but KIP-535
>> should be
>>>>>>> updated. I'll follow up on the other KIP thread to raise this
>> point.
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 1/11/20 12:26 AM, Vito Jeng wrote:
>>>>>>>> Hi, Matthias & Vinoth,
>>>>>>>>
>>>>>>>> Thanks for the feedback.
>>>>>>>>
>>>>>>>>> What is still unclear to me is, what we gain by having both
>>>>>>>>> `StreamsNotRunningException` and
>>>>> `StateStoreNotAvailableException`. Both
>>>>>>>>> exception are thrown when KafkaStreams is in state
>>>>> PENDING_SHUTDOWN /
>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>> if the
>>>>>>>>> state store is closed on not -- I can't query it anyway? Maybe
>> I
>>>>> miss
>>>>>>>>> something thought?
>>>>>>>>
>>>>>>>> Yes, both `StreamsNotRunningException` and
>>>>>>>> `StateStoreNotAvailableException` are fatal exception.
>>>>>>>> But `StateStoreNotAvailableException` is fatal exception about
>> state
>>>>>>> store
>>>>>>>> related.
>>>>>>>> I think it would be helpful that if user need to distinguish
>> these
>>>>> two
>>>>>>>> different case to handle it.
>>>>>>>>
>>>>>>>> I'm not very sure, does that make sense?
>>>>>>>>
>>>>>>>>
>>>>>>>> ---
>>>>>>>> Vito
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
>> vinoth@apache.org>
>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> +1 on merging `StreamsNotRunningException` and
>>>>>>>>> `StateStoreNotAvailableException`, both exceptions are fatal
>>>>> anyway. IMO
>>>>>>>>> its best to have these exceptions be about the state store
>> (and not
>>>>>>> streams
>>>>>>>>> state), to easier understanding.
>>>>>>>>>
>>>>>>>>> Additionally, KIP-535 allows for querying of state stores in
>>>>> rebalancing
>>>>>>>>> state. So do we need the StreamsRebalancingException?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 2020/01/09 03:38:11, "Matthias J. Sax" <
>> matthias@confluent.io>
>>>>>>> wrote:
>>>>>>>>>> Sorry that I dropped the ball on this...
>>>>>>>>>>
>>>>>>>>>> Thanks for updating the KIP. Overall LGTM now. Feel free to
>> start
>>>>> a
>>>>>>> VOTE
>>>>>>>>>> thread.
>>>>>>>>>>
>>>>>>>>>> What is still unclear to me is, what we gain by having both
>>>>>>>>>> `StreamsNotRunningException` and
>>>>> `StateStoreNotAvailableException`.
>>>>>>> Both
>>>>>>>>>> exception are thrown when KafkaStreams is in state
>>>>> PENDING_SHUTDOWN /
>>>>>>>>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
>> if
>>>>> the
>>>>>>>>>> state store is closed on not -- I can't query it anyway?
>> Maybe I
>>>>> miss
>>>>>>>>>> something thought?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>>>>>>>>>>> Sorry for the late reply, thanks for the review.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> About `StateStoreMigratedException`:
>>>>>>>>>>>>
>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
>>>>> might be
>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
>>>>> back to
>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>> store
>>>>>>> handle.
>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>>>>> `RUNNING`
>>>>>>>>> state.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Thank you point this, already updated.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Why do we need to distinguish between
>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> `KafkaStreamsNotRunningException` may be caused by various
>>>>> reasons, I
>>>>>>>>> think
>>>>>>>>>>> it would be helpful that the
>>>>>>>>>>> user can distinguish whether it is caused by the state store
>>>>> closed.
>>>>>>>>>>> (Maybe I am wrong...)
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams` instance
>> and
>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
>> the
>>>>>>>>> instance,
>>>>>>>>>>>> because that is the level of granularity in which we
>>>>> enable/disable
>>>>>>>>> IQ atm.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Totally agree. Do you mean the naming of state store
>> exceptions?
>>>>>>>>>>> I don't have special reason to distinguish these two.
>>>>>>>>>>> Your suggestion look more reasonable for the exception
>> naming.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
>> user
>>>>> need
>>>>>>>>> to
>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
>>>>> handle is
>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
>>>>>>> difference
>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>> "blind"
>>>>> retries
>>>>>>>>>>>> that either resolve (if the store is still on the same
>> instance
>>>>> after
>>>>>>>>>>>> rebalancing finishes, or changes to
>>>>> `StateStoreMigratedException` if
>>>>>>>>> the
>>>>>>>>>>>> store was migrated away during rebalancing).
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Nice, it's great! Thank you.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> The KIP already updated, please take a look. :)
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
>>>>>>> matthias@confluent.io
>>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Any update on this KIP?
>>>>>>>>>>>>
>>>>>>>>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
>> busy.
>>>>>>>>>>>>>
>>>>>>>>>>>>> About `StateStoreMigratedException`:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Why is it only thrown if the state is REBALANCING? A store
>>>>> might be
>>>>>>>>>>>>> migrated during a rebalance, and Kafka Streams might resume
>>>>> back to
>>>>>>>>>>>>> RUNNING state and afterward somebody tries to use an old
>> store
>>>>>>>>> handle.
>>>>>>>>>>>>> Also, if state is REBALANCING, should we throw
>>>>>>>>>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>>>>>>>>>> `StateStoreMigratedException` does only make sense during
>>>>> `RUNNING`
>>>>>>>>>>>> state.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Why do we need to distinguish between
>>>>>>>>> `KafkaStreamsNotRunningException`
>>>>>>>>>>>>> and `StateStoreNotAvailableException`?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Last, why do we distinguish between `KafkaStreams`
>> instance and
>>>>>>>>>>>>> `StreamsThread`? To me, it seems we should always refer to
>> the
>>>>>>>>> instance,
>>>>>>>>>>>>> because that is the level of granularity in which we
>>>>> enable/disable
>>>>>>>>> IQ
>>>>>>>>>>>> atm.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Last, for `StateStoreMigratedException`, I would add that a
>>>>> user
>>>>>>>>> need to
>>>>>>>>>>>>> rediscover the store and cannot blindly retry as the store
>>>>> handle is
>>>>>>>>>>>>> invalid and a new store handle must be retrieved. That is a
>>>>>>>>> difference
>>>>>>>>>>>>> to `StreamThreadRebalancingException` that allows for
>> "blind"
>>>>>>> retries
>>>>>>>>>>>>> that either resolve (if the store is still on the same
>> instance
>>>>>>> after
>>>>>>>>>>>>> rebalancing finishes, or changes to
>>>>> `StateStoreMigratedException` if
>>>>>>>>> the
>>>>>>>>>>>>> store was migrated away during rebalancing).
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>>>>>>>>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9>
>>>>> <https://shorturl.at/CDNT9>
>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>> <https://shorturl.at/CDNT9>
>>>>>>>>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Please use the following instead:
>> https://shorturl.at/bkKQU
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ---
>>>>>>>>>>>>>> Vito
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
>>>>> vito@is-land.com.tw>
>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks, Matthias!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> About `StreamThreadNotStartedException`:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thank you for explanation. I agree with your opinion.
>>>>>>>>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>>>>>>>>>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> For the case that corresponding thread crashes after we
>>>>> handed out
>>>>>>>>> the
>>>>>>>>>>>>>>> store handle. We may throw
>> `KafkaStreamsNotRunningException`
>>>>> or
>>>>>>>>>>>>>>> `StateStoreMigratedException`.
>>>>>>>>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
>> not
>>>>>>>>> running(
>>>>>>>>>>>>>>> https://shorturl.at/CDNT9) or throw
>>>>> `StateStoreMigratedException`
>>>>>>>>> when
>>>>>>>>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
>> we
>>>>> do not
>>>>>>>>> need
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> add a new type for this case. Does that make sense?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> About `KafkaStreamsNotRunningException` vs
>>>>>>>>>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I understand your point. I rename
>>>>>>>>> `StreamThreadNotRunningException` to
>>>>>>>>>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> About check unknown state store names:
>>>>>>>>>>>>>>> Thank you for the hint. I add a new type
>>>>>>>>> `UnknownStateStoreException`
>>>>>>>>>>>> for
>>>>>>>>>>>>>>> this case.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Also, we should still have fatal exception
>>>>>>>>>>>>>>> `StateStoreNotAvailableException`? Not sure why you
>> remove
>>>>> it?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thank you point this, already add it again.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The KIP already updated, please take a look.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> ---
>>>>>>>>>>>>>>> Vito
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Bill Bejeck <bb...@gmail.com>.
Vito,

Thanks for the updates, the KIP LGTM.

-Bill

On Wed, Jan 15, 2020 at 11:31 PM John Roesler <vv...@apache.org> wrote:

> Hi Vito,
>
> Haha, your archive game is on point!
>
> What Matthias said in that email is essentially what I figured was the
> rationale. It makes sense, but the point I was making is that this really
> doesn’t seem like a good way to structure a production app. On the other
> hand, considering the exception fatal has a good chance of avoiding a
> frustrating debug session if you just forgot to call start.
>
> Nevertheless, if we omit the categorization, it’s moot.
>
> It would be easy to add a categorization layer later if we want it, but
> not very easy to change it if we get it wrong.
>
> Thanks for your consideration!
> -John
>
> On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
> > Hi John,
> >
> > About `StreamsNotStartedException is strange` --
> > The original idea came from Matthias, two years ago. :)
> > You can reference here:
> >
> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> >
> > About omitting the categorization --
> > It looks reasonable. I'm fine with omitting the categorization but not
> very
> > sure it is a good choice.
> > Does any other folks provide opinion?
> >
> >
> > Hi, folks,
> >
> > Just update the KIP-216, please take a look.
> >
> > ---
> > Vito
> >
> >
> > On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> >
> > >
> > > Hi, folks,
> > >
> > > Thank you suggestion, really appreciate it. :)
> > > I understand your concern. I'll merge StreamsNotRunningException and
> > > StateStoreNotAvailableException.
> > >
> > >
> > > ---
> > > Vito
> > >
> > >
> > > On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org>
> wrote:
> > >
> > >> Hey Vito,
> > >>
> > >> Yes, thanks for the KIP. Sorry the discussion has been so long.
> > >> Hopefully, we can close it out soon.
> > >>
> > >> I agree we can drop StreamsNotRunningException in favor of
> > >> just StateStoreNotAvailableException.
> > >>
> > >> Unfortunately, I have some higher-level concerns. The value
> > >> of these exceptions is that they tell you how to handle the
> > >> various situations that can arise while querying a distributed
> > >> data store.
> > >>
> > >> Ideally, as a caller, I should be able to just catch "retriable" or
> > >> "fatal" and handle them appropriately. Otherwise, there's no
> > >> point in having categories, and we should just have all the
> > >> exceptions extend InvalidStateStoreException.
> > >>
> > >> Presently, it's not possible to tell from just the
> > >> "retriable"/"fatal" distinction what to do. You  can tell
> > >> from the descriptions of the various exceptions. E.g.:
> > >>
> > >> Retriable:
> > >>  * StreamsRebalancingException: the exact same call
> > >>     should just be retried until the rebalance is complete
> > >>  * StateStoreMigratedException: the store handle is
> > >>     now invalid, so you need to re-discover the instance
> > >>     and get a new handle on that instance. In other words,
> > >>     the query itself may be valid, but the particular method
> > >>     invocation on this particular instance has encountered
> > >>     a fatal exception.
> > >>
> > >> Fatal:
> > >>  * UnknownStateStoreException: this is truly fatal. No amount
> > >>     of retrying or re-discovering is going to get you a handle on a
> > >>     store that doesn't exist in the cluster.
> > >>  * StateStoreNotAvailableException: this is actually recoverable,
> > >>     since the store might exist in the cluster, but isn't available on
> > >>     this particular instance (which is shut down or whatever).
> > >>
> > >> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> > >> with omitting the categorization and just having 5 subclasses
> > >> of InvalidStateStoreException. Each of them would tell you
> > >> how to handle them, and it's not too many to really
> > >> understand and handle each one.
> > >>
> > >> If you really want to have a middle tier, I'd recommend:
> > >> * RetryableStateStoreException: the exact same call
> > >>     should be repeated.
> > >> * RecoverableStateStoreException: the store handle
> > >>     should be discarded and the caller should re-discover
> > >>     the location of the store and repeat the query on the
> > >>     correct instance.
> > >> * FatalStateStoreException: the query/request is totally
> > >>     invalid and will never succeed.
> > >>
> > >> However, attempting to categorize the proposed exceptions
> > >> reveals even problems with this categorization:
> > >> Retriable:
> > >> * StreamsRebalancingException
> > >> Recoverable:
> > >> * StateStoreMigratedException
> > >> * StreamsNotRunningException
> > >> Fatal:
> > >> * UnknownStateStoreException
> > >>
> > >> But StreamsNotStartedException is strange... It means that
> > >> one code path got a handle on a specific KafkaStreams object
> > >> instance and sent it a query before another code path
> > >> invoked the start() method on the exact same object instance.
> > >> It seems like the most likely scenario is that whoever wrote
> > >> the program just forgot to call start() before querying, in
> > >> which case, retrying isn't going to help, and a fatal exception
> > >> is more appropriate. I.e., it sounds like a "first 15 minutes
> > >> experience" problem, and making it fatal would be more
> > >> helpful. Even in a production context, there's no reason not
> > >> to sequence your application startup such that you don't
> > >> accept queries until after Streams is started. Thus, I guess
> > >> I'd categorize it under "fatal".
> > >>
> > >> Regardless of whether you make it fatal or retriable, you'd
> > >> still have a whole category with only one exception in it,
> > >> and the other two categories only have two exceptions.
> > >> Plus, as you pointed out in the KIP, you can't get all
> > >> exceptions in all cases anyway:
> > >> * store() can only throw NotStarted, NotRunning,
> > >>     and Unknown
> > >> * actual store queries can only throw Rebalancing,
> > >>     Migrated, and NotRunning
> > >>
> > >> Thus, in practice also, there are exactly three categories
> > >> and also exactly three exception types. It doesn't seem
> > >> like there's a great advantage to the categories here. To
> > >> avoid the categorization problem and also to clarify what
> > >> exceptions can actually be thrown in different circumstances,
> > >> it seems like we should just:
> > >> * get rid of the middle tier and make all the exceptions
> > >>     extend InvalidStateStoreException
> > >> * drop StateStoreNotAvailableException in favor of
> > >>     StreamsNotRunningException
> > >> * clearly document on all public methods which exceptions
> > >>     need to be handled
> > >>
> > >> How do you feel about this?
> > >> Thanks,
> > >> -John
> > >>
> > >> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> > >> > Thanks for KIP Vito.
> > >> >
> > >> > Overall the KIP LGTM, but I'd have to agree with others on merging
> the
> > >> > `StreamsNotRunningException` and `StateStoreNotAvailableException`
> > >> classes.
> > >> >
> > >> > Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> > >> > NOT_RUNNING || ERROR` I'm not even sure how we could distinguish
> when to
> > >> > use the different
> > >> > exceptions.  Maybe a good middle ground would be to have a detailed
> > >> > exception message.
> > >> >
> > >> > The KIP freeze is close, so I think if we can agree on this, we can
> > >> wrap up
> > >> > the voting soon.
> > >> >
> > >> > Thanks,
> > >> > Bill
> > >> >
> > >> > On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <
> matthias@confluent.io>
> > >> > wrote:
> > >> >
> > >> > > Vito,
> > >> > >
> > >> > > It's still unclear to me what the advantage is, to have both
> > >> > > `StreamsNotRunningException` and
> `StateStoreNotAvailableException`?
> > >> > >
> > >> > > For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING /
> ERROR`
> > >> > > and thus, for a user point of view, why does it matter if the
> store is
> > >> > > closed on not? I don't understand why/how this information would
> be
> > >> > > useful? Do you have a concrete example in mind how a user would
> react
> > >> > > differently to both exceptions?
> > >> > >
> > >> > >
> > >> > > @Vinoth: about `StreamsRebalancingException` -- to me, it seems
> best
> > >> to
> > >> > > actually do this on a per-query basis, ie, have an overload
> > >> > > `KafkaStreams#store(...)` that takes a boolean flag that allow to
> > >> > > _disable_ the exception and opt-in to query a active store during
> > >> > > recovery. However, as KIP-535 actually introduces this change in
> > >> > > behavior, I think KIP-216 should not cover this, but KIP-535
> should be
> > >> > > updated. I'll follow up on the other KIP thread to raise this
> point.
> > >> > >
> > >> > >
> > >> > > -Matthias
> > >> > >
> > >> > > On 1/11/20 12:26 AM, Vito Jeng wrote:
> > >> > > > Hi, Matthias & Vinoth,
> > >> > > >
> > >> > > > Thanks for the feedback.
> > >> > > >
> > >> > > >> What is still unclear to me is, what we gain by having both
> > >> > > >> `StreamsNotRunningException` and
> > >> `StateStoreNotAvailableException`. Both
> > >> > > >> exception are thrown when KafkaStreams is in state
> > >> PENDING_SHUTDOWN /
> > >> > > >> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> if the
> > >> > > >> state store is closed on not -- I can't query it anyway? Maybe
> I
> > >> miss
> > >> > > >> something thought?
> > >> > > >
> > >> > > > Yes, both `StreamsNotRunningException` and
> > >> > > > `StateStoreNotAvailableException` are fatal exception.
> > >> > > > But `StateStoreNotAvailableException` is fatal exception about
> state
> > >> > > store
> > >> > > > related.
> > >> > > > I think it would be helpful that if user need to distinguish
> these
> > >> two
> > >> > > > different case to handle it.
> > >> > > >
> > >> > > > I'm not very sure, does that make sense?
> > >> > > >
> > >> > > >
> > >> > > > ---
> > >> > > > Vito
> > >> > > >
> > >> > > >
> > >> > > > On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <
> vinoth@apache.org>
> > >> > > wrote:
> > >> > > >
> > >> > > >> +1 on merging `StreamsNotRunningException` and
> > >> > > >> `StateStoreNotAvailableException`, both exceptions are fatal
> > >> anyway. IMO
> > >> > > >> its best to have these exceptions be about the state store
> (and not
> > >> > > streams
> > >> > > >> state), to easier understanding.
> > >> > > >>
> > >> > > >> Additionally, KIP-535 allows for querying of state stores in
> > >> rebalancing
> > >> > > >> state. So do we need the StreamsRebalancingException?
> > >> > > >>
> > >> > > >>
> > >> > > >> On 2020/01/09 03:38:11, "Matthias J. Sax" <
> matthias@confluent.io>
> > >> > > wrote:
> > >> > > >>> Sorry that I dropped the ball on this...
> > >> > > >>>
> > >> > > >>> Thanks for updating the KIP. Overall LGTM now. Feel free to
> start
> > >> a
> > >> > > VOTE
> > >> > > >>> thread.
> > >> > > >>>
> > >> > > >>> What is still unclear to me is, what we gain by having both
> > >> > > >>> `StreamsNotRunningException` and
> > >> `StateStoreNotAvailableException`.
> > >> > > Both
> > >> > > >>> exception are thrown when KafkaStreams is in state
> > >> PENDING_SHUTDOWN /
> > >> > > >>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know
> if
> > >> the
> > >> > > >>> state store is closed on not -- I can't query it anyway?
> Maybe I
> > >> miss
> > >> > > >>> something thought?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> -Matthias
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> > >> > > >>>> Sorry for the late reply, thanks for the review.
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>>> About `StateStoreMigratedException`:
> > >> > > >>>>>
> > >> > > >>>>> Why is it only thrown if the state is REBALANCING? A store
> > >> might be
> > >> > > >>>>> migrated during a rebalance, and Kafka Streams might resume
> > >> back to
> > >> > > >>>>> RUNNING state and afterward somebody tries to use an old
> store
> > >> > > handle.
> > >> > > >>>>> Also, if state is REBALANCING, should we throw
> > >> > > >>>>> `StreamThreadRebalancingException`? Hence, I think
> > >> > > >>>>> `StateStoreMigratedException` does only make sense during
> > >> `RUNNING`
> > >> > > >> state.
> > >> > > >>>>>
> > >> > > >>>>
> > >> > > >>>> Thank you point this, already updated.
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> Why do we need to distinguish between
> > >> > > `KafkaStreamsNotRunningException`
> > >> > > >>>>> and `StateStoreNotAvailableException`?
> > >> > > >>>>>
> > >> > > >>>>
> > >> > > >>>> `KafkaStreamsNotRunningException` may be caused by various
> > >> reasons, I
> > >> > > >> think
> > >> > > >>>> it would be helpful that the
> > >> > > >>>> user can distinguish whether it is caused by the state store
> > >> closed.
> > >> > > >>>> (Maybe I am wrong...)
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> Last, why do we distinguish between `KafkaStreams` instance
> and
> > >> > > >>>>> `StreamsThread`? To me, it seems we should always refer to
> the
> > >> > > >> instance,
> > >> > > >>>>> because that is the level of granularity in which we
> > >> enable/disable
> > >> > > >> IQ atm.
> > >> > > >>>>>
> > >> > > >>>>
> > >> > > >>>> Totally agree. Do you mean the naming of state store
> exceptions?
> > >> > > >>>> I don't have special reason to distinguish these two.
> > >> > > >>>> Your suggestion look more reasonable for the exception
> naming.
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> Last, for `StateStoreMigratedException`, I would add that a
> user
> > >> need
> > >> > > >> to
> > >> > > >>>>> rediscover the store and cannot blindly retry as the store
> > >> handle is
> > >> > > >>>>> invalid and a new store handle must be retrieved. That is a
> > >> > > difference
> > >> > > >>>>> to `StreamThreadRebalancingException` that allows for
> "blind"
> > >> retries
> > >> > > >>>>> that either resolve (if the store is still on the same
> instance
> > >> after
> > >> > > >>>>> rebalancing finishes, or changes to
> > >> `StateStoreMigratedException` if
> > >> > > >> the
> > >> > > >>>>> store was migrated away during rebalancing).
> > >> > > >>>>>
> > >> > > >>>>
> > >> > > >>>> Nice, it's great! Thank you.
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> The KIP already updated, please take a look. :)
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> > >> > > matthias@confluent.io
> > >> > > >>>
> > >> > > >>>> wrote:
> > >> > > >>>>
> > >> > > >>>>> Any update on this KIP?
> > >> > > >>>>>
> > >> > > >>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> > >> > > >>>>>> Sorry for the late reply. The 2.4 deadline kept us quite
> busy.
> > >> > > >>>>>>
> > >> > > >>>>>> About `StateStoreMigratedException`:
> > >> > > >>>>>>
> > >> > > >>>>>> Why is it only thrown if the state is REBALANCING? A store
> > >> might be
> > >> > > >>>>>> migrated during a rebalance, and Kafka Streams might resume
> > >> back to
> > >> > > >>>>>> RUNNING state and afterward somebody tries to use an old
> store
> > >> > > >> handle.
> > >> > > >>>>>> Also, if state is REBALANCING, should we throw
> > >> > > >>>>>> `StreamThreadRebalancingException`? Hence, I think
> > >> > > >>>>>> `StateStoreMigratedException` does only make sense during
> > >> `RUNNING`
> > >> > > >>>>> state.
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> Why do we need to distinguish between
> > >> > > >> `KafkaStreamsNotRunningException`
> > >> > > >>>>>> and `StateStoreNotAvailableException`?
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> Last, why do we distinguish between `KafkaStreams`
> instance and
> > >> > > >>>>>> `StreamsThread`? To me, it seems we should always refer to
> the
> > >> > > >> instance,
> > >> > > >>>>>> because that is the level of granularity in which we
> > >> enable/disable
> > >> > > >> IQ
> > >> > > >>>>> atm.
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> Last, for `StateStoreMigratedException`, I would add that a
> > >> user
> > >> > > >> need to
> > >> > > >>>>>> rediscover the store and cannot blindly retry as the store
> > >> handle is
> > >> > > >>>>>> invalid and a new store handle must be retrieved. That is a
> > >> > > >> difference
> > >> > > >>>>>> to `StreamThreadRebalancingException` that allows for
> "blind"
> > >> > > retries
> > >> > > >>>>>> that either resolve (if the store is still on the same
> instance
> > >> > > after
> > >> > > >>>>>> rebalancing finishes, or changes to
> > >> `StateStoreMigratedException` if
> > >> > > >> the
> > >> > > >>>>>> store was migrated away during rebalancing).
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> -Matthias
> > >> > > >>>>>>
> > >> > > >>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> > >> > > >>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> > >> <https://shorturl.at/CDNT9>
> > >> > > <https://shorturl.at/CDNT9>
> > >> > > >> <https://shorturl.at/CDNT9>
> > >> > > >>>>> <https://shorturl.at/CDNT9>
> > >> > > >>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Please use the following instead:
> https://shorturl.at/bkKQU
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>> ---
> > >> > > >>>>>>> Vito
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
> > >> vito@is-land.com.tw>
> > >> > > >> wrote:
> > >> > > >>>>>>>
> > >> > > >>>>>>>> Thanks, Matthias!
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>> About `StreamThreadNotStartedException`:
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> Thank you for explanation. I agree with your opinion.
> > >> > > >>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> > >> > > >>>>>>>> `StreamThreadNotStartedException`.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> For the case that corresponding thread crashes after we
> > >> handed out
> > >> > > >> the
> > >> > > >>>>>>>> store handle. We may throw
> `KafkaStreamsNotRunningException`
> > >> or
> > >> > > >>>>>>>> `StateStoreMigratedException`.
> > >> > > >>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> > >> > > >>>>>>>> `KafkaStreamsNotRunningException` when stream thread is
> not
> > >> > > >> running(
> > >> > > >>>>>>>> https://shorturl.at/CDNT9) or throw
> > >> `StateStoreMigratedException`
> > >> > > >> when
> > >> > > >>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think
> we
> > >> do not
> > >> > > >> need
> > >> > > >>>>> to
> > >> > > >>>>>>>> add a new type for this case. Does that make sense?
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>> About `KafkaStreamsNotRunningException` vs
> > >> > > >>>>>>>> `StreamThreadNotRunningException`:
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> I understand your point. I rename
> > >> > > >> `StreamThreadNotRunningException` to
> > >> > > >>>>>>>> `KafkaStreamsNotRunningException`.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> About check unknown state store names:
> > >> > > >>>>>>>> Thank you for the hint. I add a new type
> > >> > > >> `UnknownStateStoreException`
> > >> > > >>>>> for
> > >> > > >>>>>>>> this case.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>> Also, we should still have fatal exception
> > >> > > >>>>>>>> `StateStoreNotAvailableException`? Not sure why you
> remove
> > >> it?
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> Thank you point this, already add it again.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> The KIP already updated, please take a look.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> ---
> > >> > > >>>>>>>> Vito
> > >> > > >>>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>
> > >> > > >>>
> > >> > > >>>
> > >> > > >>
> > >> > > >
> > >> > >
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by John Roesler <vv...@apache.org>.
Hi Vito,

Haha, your archive game is on point!

What Matthias said in that email is essentially what I figured was the rationale. It makes sense, but the point I was making is that this really doesn’t seem like a good way to structure a production app. On the other hand, considering the exception fatal has a good chance of avoiding a frustrating debug session if you just forgot to call start. 

Nevertheless, if we omit the categorization, it’s moot.

It would be easy to add a categorization layer later if we want it, but not very easy to change it if we get it wrong. 

Thanks for your consideration!
-John

On Wed, Jan 15, 2020, at 21:14, Vito Jeng wrote:
> Hi John,
> 
> About `StreamsNotStartedException is strange` --
> The original idea came from Matthias, two years ago. :)
> You can reference here:
> https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e
> 
> About omitting the categorization --
> It looks reasonable. I'm fine with omitting the categorization but not very
> sure it is a good choice.
> Does any other folks provide opinion?
> 
> 
> Hi, folks,
> 
> Just update the KIP-216, please take a look.
> 
> ---
> Vito
> 
> 
> On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> 
> >
> > Hi, folks,
> >
> > Thank you suggestion, really appreciate it. :)
> > I understand your concern. I'll merge StreamsNotRunningException and
> > StateStoreNotAvailableException.
> >
> >
> > ---
> > Vito
> >
> >
> > On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org> wrote:
> >
> >> Hey Vito,
> >>
> >> Yes, thanks for the KIP. Sorry the discussion has been so long.
> >> Hopefully, we can close it out soon.
> >>
> >> I agree we can drop StreamsNotRunningException in favor of
> >> just StateStoreNotAvailableException.
> >>
> >> Unfortunately, I have some higher-level concerns. The value
> >> of these exceptions is that they tell you how to handle the
> >> various situations that can arise while querying a distributed
> >> data store.
> >>
> >> Ideally, as a caller, I should be able to just catch "retriable" or
> >> "fatal" and handle them appropriately. Otherwise, there's no
> >> point in having categories, and we should just have all the
> >> exceptions extend InvalidStateStoreException.
> >>
> >> Presently, it's not possible to tell from just the
> >> "retriable"/"fatal" distinction what to do. You  can tell
> >> from the descriptions of the various exceptions. E.g.:
> >>
> >> Retriable:
> >>  * StreamsRebalancingException: the exact same call
> >>     should just be retried until the rebalance is complete
> >>  * StateStoreMigratedException: the store handle is
> >>     now invalid, so you need to re-discover the instance
> >>     and get a new handle on that instance. In other words,
> >>     the query itself may be valid, but the particular method
> >>     invocation on this particular instance has encountered
> >>     a fatal exception.
> >>
> >> Fatal:
> >>  * UnknownStateStoreException: this is truly fatal. No amount
> >>     of retrying or re-discovering is going to get you a handle on a
> >>     store that doesn't exist in the cluster.
> >>  * StateStoreNotAvailableException: this is actually recoverable,
> >>     since the store might exist in the cluster, but isn't available on
> >>     this particular instance (which is shut down or whatever).
> >>
> >> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> >> with omitting the categorization and just having 5 subclasses
> >> of InvalidStateStoreException. Each of them would tell you
> >> how to handle them, and it's not too many to really
> >> understand and handle each one.
> >>
> >> If you really want to have a middle tier, I'd recommend:
> >> * RetryableStateStoreException: the exact same call
> >>     should be repeated.
> >> * RecoverableStateStoreException: the store handle
> >>     should be discarded and the caller should re-discover
> >>     the location of the store and repeat the query on the
> >>     correct instance.
> >> * FatalStateStoreException: the query/request is totally
> >>     invalid and will never succeed.
> >>
> >> However, attempting to categorize the proposed exceptions
> >> reveals even problems with this categorization:
> >> Retriable:
> >> * StreamsRebalancingException
> >> Recoverable:
> >> * StateStoreMigratedException
> >> * StreamsNotRunningException
> >> Fatal:
> >> * UnknownStateStoreException
> >>
> >> But StreamsNotStartedException is strange... It means that
> >> one code path got a handle on a specific KafkaStreams object
> >> instance and sent it a query before another code path
> >> invoked the start() method on the exact same object instance.
> >> It seems like the most likely scenario is that whoever wrote
> >> the program just forgot to call start() before querying, in
> >> which case, retrying isn't going to help, and a fatal exception
> >> is more appropriate. I.e., it sounds like a "first 15 minutes
> >> experience" problem, and making it fatal would be more
> >> helpful. Even in a production context, there's no reason not
> >> to sequence your application startup such that you don't
> >> accept queries until after Streams is started. Thus, I guess
> >> I'd categorize it under "fatal".
> >>
> >> Regardless of whether you make it fatal or retriable, you'd
> >> still have a whole category with only one exception in it,
> >> and the other two categories only have two exceptions.
> >> Plus, as you pointed out in the KIP, you can't get all
> >> exceptions in all cases anyway:
> >> * store() can only throw NotStarted, NotRunning,
> >>     and Unknown
> >> * actual store queries can only throw Rebalancing,
> >>     Migrated, and NotRunning
> >>
> >> Thus, in practice also, there are exactly three categories
> >> and also exactly three exception types. It doesn't seem
> >> like there's a great advantage to the categories here. To
> >> avoid the categorization problem and also to clarify what
> >> exceptions can actually be thrown in different circumstances,
> >> it seems like we should just:
> >> * get rid of the middle tier and make all the exceptions
> >>     extend InvalidStateStoreException
> >> * drop StateStoreNotAvailableException in favor of
> >>     StreamsNotRunningException
> >> * clearly document on all public methods which exceptions
> >>     need to be handled
> >>
> >> How do you feel about this?
> >> Thanks,
> >> -John
> >>
> >> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> >> > Thanks for KIP Vito.
> >> >
> >> > Overall the KIP LGTM, but I'd have to agree with others on merging the
> >> > `StreamsNotRunningException` and `StateStoreNotAvailableException`
> >> classes.
> >> >
> >> > Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> >> > NOT_RUNNING || ERROR` I'm not even sure how we could distinguish when to
> >> > use the different
> >> > exceptions.  Maybe a good middle ground would be to have a detailed
> >> > exception message.
> >> >
> >> > The KIP freeze is close, so I think if we can agree on this, we can
> >> wrap up
> >> > the voting soon.
> >> >
> >> > Thanks,
> >> > Bill
> >> >
> >> > On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <ma...@confluent.io>
> >> > wrote:
> >> >
> >> > > Vito,
> >> > >
> >> > > It's still unclear to me what the advantage is, to have both
> >> > > `StreamsNotRunningException` and `StateStoreNotAvailableException`?
> >> > >
> >> > > For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING / ERROR`
> >> > > and thus, for a user point of view, why does it matter if the store is
> >> > > closed on not? I don't understand why/how this information would be
> >> > > useful? Do you have a concrete example in mind how a user would react
> >> > > differently to both exceptions?
> >> > >
> >> > >
> >> > > @Vinoth: about `StreamsRebalancingException` -- to me, it seems best
> >> to
> >> > > actually do this on a per-query basis, ie, have an overload
> >> > > `KafkaStreams#store(...)` that takes a boolean flag that allow to
> >> > > _disable_ the exception and opt-in to query a active store during
> >> > > recovery. However, as KIP-535 actually introduces this change in
> >> > > behavior, I think KIP-216 should not cover this, but KIP-535 should be
> >> > > updated. I'll follow up on the other KIP thread to raise this point.
> >> > >
> >> > >
> >> > > -Matthias
> >> > >
> >> > > On 1/11/20 12:26 AM, Vito Jeng wrote:
> >> > > > Hi, Matthias & Vinoth,
> >> > > >
> >> > > > Thanks for the feedback.
> >> > > >
> >> > > >> What is still unclear to me is, what we gain by having both
> >> > > >> `StreamsNotRunningException` and
> >> `StateStoreNotAvailableException`. Both
> >> > > >> exception are thrown when KafkaStreams is in state
> >> PENDING_SHUTDOWN /
> >> > > >> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> >> > > >> state store is closed on not -- I can't query it anyway? Maybe I
> >> miss
> >> > > >> something thought?
> >> > > >
> >> > > > Yes, both `StreamsNotRunningException` and
> >> > > > `StateStoreNotAvailableException` are fatal exception.
> >> > > > But `StateStoreNotAvailableException` is fatal exception about state
> >> > > store
> >> > > > related.
> >> > > > I think it would be helpful that if user need to distinguish these
> >> two
> >> > > > different case to handle it.
> >> > > >
> >> > > > I'm not very sure, does that make sense?
> >> > > >
> >> > > >
> >> > > > ---
> >> > > > Vito
> >> > > >
> >> > > >
> >> > > > On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <vi...@apache.org>
> >> > > wrote:
> >> > > >
> >> > > >> +1 on merging `StreamsNotRunningException` and
> >> > > >> `StateStoreNotAvailableException`, both exceptions are fatal
> >> anyway. IMO
> >> > > >> its best to have these exceptions be about the state store (and not
> >> > > streams
> >> > > >> state), to easier understanding.
> >> > > >>
> >> > > >> Additionally, KIP-535 allows for querying of state stores in
> >> rebalancing
> >> > > >> state. So do we need the StreamsRebalancingException?
> >> > > >>
> >> > > >>
> >> > > >> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io>
> >> > > wrote:
> >> > > >>> Sorry that I dropped the ball on this...
> >> > > >>>
> >> > > >>> Thanks for updating the KIP. Overall LGTM now. Feel free to start
> >> a
> >> > > VOTE
> >> > > >>> thread.
> >> > > >>>
> >> > > >>> What is still unclear to me is, what we gain by having both
> >> > > >>> `StreamsNotRunningException` and
> >> `StateStoreNotAvailableException`.
> >> > > Both
> >> > > >>> exception are thrown when KafkaStreams is in state
> >> PENDING_SHUTDOWN /
> >> > > >>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if
> >> the
> >> > > >>> state store is closed on not -- I can't query it anyway? Maybe I
> >> miss
> >> > > >>> something thought?
> >> > > >>>
> >> > > >>>
> >> > > >>> -Matthias
> >> > > >>>
> >> > > >>>
> >> > > >>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> >> > > >>>> Sorry for the late reply, thanks for the review.
> >> > > >>>>
> >> > > >>>>
> >> > > >>>>> About `StateStoreMigratedException`:
> >> > > >>>>>
> >> > > >>>>> Why is it only thrown if the state is REBALANCING? A store
> >> might be
> >> > > >>>>> migrated during a rebalance, and Kafka Streams might resume
> >> back to
> >> > > >>>>> RUNNING state and afterward somebody tries to use an old store
> >> > > handle.
> >> > > >>>>> Also, if state is REBALANCING, should we throw
> >> > > >>>>> `StreamThreadRebalancingException`? Hence, I think
> >> > > >>>>> `StateStoreMigratedException` does only make sense during
> >> `RUNNING`
> >> > > >> state.
> >> > > >>>>>
> >> > > >>>>
> >> > > >>>> Thank you point this, already updated.
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> Why do we need to distinguish between
> >> > > `KafkaStreamsNotRunningException`
> >> > > >>>>> and `StateStoreNotAvailableException`?
> >> > > >>>>>
> >> > > >>>>
> >> > > >>>> `KafkaStreamsNotRunningException` may be caused by various
> >> reasons, I
> >> > > >> think
> >> > > >>>> it would be helpful that the
> >> > > >>>> user can distinguish whether it is caused by the state store
> >> closed.
> >> > > >>>> (Maybe I am wrong...)
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> Last, why do we distinguish between `KafkaStreams` instance and
> >> > > >>>>> `StreamsThread`? To me, it seems we should always refer to the
> >> > > >> instance,
> >> > > >>>>> because that is the level of granularity in which we
> >> enable/disable
> >> > > >> IQ atm.
> >> > > >>>>>
> >> > > >>>>
> >> > > >>>> Totally agree. Do you mean the naming of state store exceptions?
> >> > > >>>> I don't have special reason to distinguish these two.
> >> > > >>>> Your suggestion look more reasonable for the exception naming.
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> Last, for `StateStoreMigratedException`, I would add that a user
> >> need
> >> > > >> to
> >> > > >>>>> rediscover the store and cannot blindly retry as the store
> >> handle is
> >> > > >>>>> invalid and a new store handle must be retrieved. That is a
> >> > > difference
> >> > > >>>>> to `StreamThreadRebalancingException` that allows for "blind"
> >> retries
> >> > > >>>>> that either resolve (if the store is still on the same instance
> >> after
> >> > > >>>>> rebalancing finishes, or changes to
> >> `StateStoreMigratedException` if
> >> > > >> the
> >> > > >>>>> store was migrated away during rebalancing).
> >> > > >>>>>
> >> > > >>>>
> >> > > >>>> Nice, it's great! Thank you.
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> The KIP already updated, please take a look. :)
> >> > > >>>>
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> >> > > matthias@confluent.io
> >> > > >>>
> >> > > >>>> wrote:
> >> > > >>>>
> >> > > >>>>> Any update on this KIP?
> >> > > >>>>>
> >> > > >>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >> > > >>>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> >> > > >>>>>>
> >> > > >>>>>> About `StateStoreMigratedException`:
> >> > > >>>>>>
> >> > > >>>>>> Why is it only thrown if the state is REBALANCING? A store
> >> might be
> >> > > >>>>>> migrated during a rebalance, and Kafka Streams might resume
> >> back to
> >> > > >>>>>> RUNNING state and afterward somebody tries to use an old store
> >> > > >> handle.
> >> > > >>>>>> Also, if state is REBALANCING, should we throw
> >> > > >>>>>> `StreamThreadRebalancingException`? Hence, I think
> >> > > >>>>>> `StateStoreMigratedException` does only make sense during
> >> `RUNNING`
> >> > > >>>>> state.
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> Why do we need to distinguish between
> >> > > >> `KafkaStreamsNotRunningException`
> >> > > >>>>>> and `StateStoreNotAvailableException`?
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> Last, why do we distinguish between `KafkaStreams` instance and
> >> > > >>>>>> `StreamsThread`? To me, it seems we should always refer to the
> >> > > >> instance,
> >> > > >>>>>> because that is the level of granularity in which we
> >> enable/disable
> >> > > >> IQ
> >> > > >>>>> atm.
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> Last, for `StateStoreMigratedException`, I would add that a
> >> user
> >> > > >> need to
> >> > > >>>>>> rediscover the store and cannot blindly retry as the store
> >> handle is
> >> > > >>>>>> invalid and a new store handle must be retrieved. That is a
> >> > > >> difference
> >> > > >>>>>> to `StreamThreadRebalancingException` that allows for "blind"
> >> > > retries
> >> > > >>>>>> that either resolve (if the store is still on the same instance
> >> > > after
> >> > > >>>>>> rebalancing finishes, or changes to
> >> `StateStoreMigratedException` if
> >> > > >> the
> >> > > >>>>>> store was migrated away during rebalancing).
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> -Matthias
> >> > > >>>>>>
> >> > > >>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >> > > >>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> >> <https://shorturl.at/CDNT9>
> >> > > <https://shorturl.at/CDNT9>
> >> > > >> <https://shorturl.at/CDNT9>
> >> > > >>>>> <https://shorturl.at/CDNT9>
> >> > > >>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> >> > > >>>>>>>
> >> > > >>>>>>> Please use the following instead: https://shorturl.at/bkKQU
> >> > > >>>>>>>
> >> > > >>>>>>>
> >> > > >>>>>>> ---
> >> > > >>>>>>> Vito
> >> > > >>>>>>>
> >> > > >>>>>>>
> >> > > >>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
> >> vito@is-land.com.tw>
> >> > > >> wrote:
> >> > > >>>>>>>
> >> > > >>>>>>>> Thanks, Matthias!
> >> > > >>>>>>>>
> >> > > >>>>>>>>> About `StreamThreadNotStartedException`:
> >> > > >>>>>>>>
> >> > > >>>>>>>> Thank you for explanation. I agree with your opinion.
> >> > > >>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >> > > >>>>>>>> `StreamThreadNotStartedException`.
> >> > > >>>>>>>>
> >> > > >>>>>>>> For the case that corresponding thread crashes after we
> >> handed out
> >> > > >> the
> >> > > >>>>>>>> store handle. We may throw `KafkaStreamsNotRunningException`
> >> or
> >> > > >>>>>>>> `StateStoreMigratedException`.
> >> > > >>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> >> > > >>>>>>>> `KafkaStreamsNotRunningException` when stream thread is not
> >> > > >> running(
> >> > > >>>>>>>> https://shorturl.at/CDNT9) or throw
> >> `StateStoreMigratedException`
> >> > > >> when
> >> > > >>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we
> >> do not
> >> > > >> need
> >> > > >>>>> to
> >> > > >>>>>>>> add a new type for this case. Does that make sense?
> >> > > >>>>>>>>
> >> > > >>>>>>>>
> >> > > >>>>>>>>> About `KafkaStreamsNotRunningException` vs
> >> > > >>>>>>>> `StreamThreadNotRunningException`:
> >> > > >>>>>>>>
> >> > > >>>>>>>> I understand your point. I rename
> >> > > >> `StreamThreadNotRunningException` to
> >> > > >>>>>>>> `KafkaStreamsNotRunningException`.
> >> > > >>>>>>>>
> >> > > >>>>>>>>
> >> > > >>>>>>>> About check unknown state store names:
> >> > > >>>>>>>> Thank you for the hint. I add a new type
> >> > > >> `UnknownStateStoreException`
> >> > > >>>>> for
> >> > > >>>>>>>> this case.
> >> > > >>>>>>>>
> >> > > >>>>>>>>
> >> > > >>>>>>>>> Also, we should still have fatal exception
> >> > > >>>>>>>> `StateStoreNotAvailableException`? Not sure why you remove
> >> it?
> >> > > >>>>>>>>
> >> > > >>>>>>>> Thank you point this, already add it again.
> >> > > >>>>>>>>
> >> > > >>>>>>>> The KIP already updated, please take a look.
> >> > > >>>>>>>>
> >> > > >>>>>>>> ---
> >> > > >>>>>>>> Vito
> >> > > >>>>>>>>
> >> > > >>>>>>>
> >> > > >>>>>>
> >> > > >>>>>
> >> > > >>>>>
> >> > > >>>>
> >> > > >>>
> >> > > >>>
> >> > > >>
> >> > > >
> >> > >
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Hi John,

About `StreamsNotStartedException is strange` --
The original idea came from Matthias, two years ago. :)
You can reference here:
https://mail-archives.apache.org/mod_mbox/kafka-dev/201806.mbox/%3c6c32083e-b63c-435b-521d-032d45cc518f@confluent.io%3e

About omitting the categorization --
It looks reasonable. I'm fine with omitting the categorization but not very
sure it is a good choice.
Does any other folks provide opinion?


Hi, folks,

Just update the KIP-216, please take a look.

---
Vito


On Thu, Jan 16, 2020 at 6:35 AM Vito Jeng <vi...@is-land.com.tw> wrote:

>
> Hi, folks,
>
> Thank you suggestion, really appreciate it. :)
> I understand your concern. I'll merge StreamsNotRunningException and
> StateStoreNotAvailableException.
>
>
> ---
> Vito
>
>
> On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org> wrote:
>
>> Hey Vito,
>>
>> Yes, thanks for the KIP. Sorry the discussion has been so long.
>> Hopefully, we can close it out soon.
>>
>> I agree we can drop StreamsNotRunningException in favor of
>> just StateStoreNotAvailableException.
>>
>> Unfortunately, I have some higher-level concerns. The value
>> of these exceptions is that they tell you how to handle the
>> various situations that can arise while querying a distributed
>> data store.
>>
>> Ideally, as a caller, I should be able to just catch "retriable" or
>> "fatal" and handle them appropriately. Otherwise, there's no
>> point in having categories, and we should just have all the
>> exceptions extend InvalidStateStoreException.
>>
>> Presently, it's not possible to tell from just the
>> "retriable"/"fatal" distinction what to do. You  can tell
>> from the descriptions of the various exceptions. E.g.:
>>
>> Retriable:
>>  * StreamsRebalancingException: the exact same call
>>     should just be retried until the rebalance is complete
>>  * StateStoreMigratedException: the store handle is
>>     now invalid, so you need to re-discover the instance
>>     and get a new handle on that instance. In other words,
>>     the query itself may be valid, but the particular method
>>     invocation on this particular instance has encountered
>>     a fatal exception.
>>
>> Fatal:
>>  * UnknownStateStoreException: this is truly fatal. No amount
>>     of retrying or re-discovering is going to get you a handle on a
>>     store that doesn't exist in the cluster.
>>  * StateStoreNotAvailableException: this is actually recoverable,
>>     since the store might exist in the cluster, but isn't available on
>>     this particular instance (which is shut down or whatever).
>>
>> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
>> with omitting the categorization and just having 5 subclasses
>> of InvalidStateStoreException. Each of them would tell you
>> how to handle them, and it's not too many to really
>> understand and handle each one.
>>
>> If you really want to have a middle tier, I'd recommend:
>> * RetryableStateStoreException: the exact same call
>>     should be repeated.
>> * RecoverableStateStoreException: the store handle
>>     should be discarded and the caller should re-discover
>>     the location of the store and repeat the query on the
>>     correct instance.
>> * FatalStateStoreException: the query/request is totally
>>     invalid and will never succeed.
>>
>> However, attempting to categorize the proposed exceptions
>> reveals even problems with this categorization:
>> Retriable:
>> * StreamsRebalancingException
>> Recoverable:
>> * StateStoreMigratedException
>> * StreamsNotRunningException
>> Fatal:
>> * UnknownStateStoreException
>>
>> But StreamsNotStartedException is strange... It means that
>> one code path got a handle on a specific KafkaStreams object
>> instance and sent it a query before another code path
>> invoked the start() method on the exact same object instance.
>> It seems like the most likely scenario is that whoever wrote
>> the program just forgot to call start() before querying, in
>> which case, retrying isn't going to help, and a fatal exception
>> is more appropriate. I.e., it sounds like a "first 15 minutes
>> experience" problem, and making it fatal would be more
>> helpful. Even in a production context, there's no reason not
>> to sequence your application startup such that you don't
>> accept queries until after Streams is started. Thus, I guess
>> I'd categorize it under "fatal".
>>
>> Regardless of whether you make it fatal or retriable, you'd
>> still have a whole category with only one exception in it,
>> and the other two categories only have two exceptions.
>> Plus, as you pointed out in the KIP, you can't get all
>> exceptions in all cases anyway:
>> * store() can only throw NotStarted, NotRunning,
>>     and Unknown
>> * actual store queries can only throw Rebalancing,
>>     Migrated, and NotRunning
>>
>> Thus, in practice also, there are exactly three categories
>> and also exactly three exception types. It doesn't seem
>> like there's a great advantage to the categories here. To
>> avoid the categorization problem and also to clarify what
>> exceptions can actually be thrown in different circumstances,
>> it seems like we should just:
>> * get rid of the middle tier and make all the exceptions
>>     extend InvalidStateStoreException
>> * drop StateStoreNotAvailableException in favor of
>>     StreamsNotRunningException
>> * clearly document on all public methods which exceptions
>>     need to be handled
>>
>> How do you feel about this?
>> Thanks,
>> -John
>>
>> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
>> > Thanks for KIP Vito.
>> >
>> > Overall the KIP LGTM, but I'd have to agree with others on merging the
>> > `StreamsNotRunningException` and `StateStoreNotAvailableException`
>> classes.
>> >
>> > Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
>> > NOT_RUNNING || ERROR` I'm not even sure how we could distinguish when to
>> > use the different
>> > exceptions.  Maybe a good middle ground would be to have a detailed
>> > exception message.
>> >
>> > The KIP freeze is close, so I think if we can agree on this, we can
>> wrap up
>> > the voting soon.
>> >
>> > Thanks,
>> > Bill
>> >
>> > On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <ma...@confluent.io>
>> > wrote:
>> >
>> > > Vito,
>> > >
>> > > It's still unclear to me what the advantage is, to have both
>> > > `StreamsNotRunningException` and `StateStoreNotAvailableException`?
>> > >
>> > > For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING / ERROR`
>> > > and thus, for a user point of view, why does it matter if the store is
>> > > closed on not? I don't understand why/how this information would be
>> > > useful? Do you have a concrete example in mind how a user would react
>> > > differently to both exceptions?
>> > >
>> > >
>> > > @Vinoth: about `StreamsRebalancingException` -- to me, it seems best
>> to
>> > > actually do this on a per-query basis, ie, have an overload
>> > > `KafkaStreams#store(...)` that takes a boolean flag that allow to
>> > > _disable_ the exception and opt-in to query a active store during
>> > > recovery. However, as KIP-535 actually introduces this change in
>> > > behavior, I think KIP-216 should not cover this, but KIP-535 should be
>> > > updated. I'll follow up on the other KIP thread to raise this point.
>> > >
>> > >
>> > > -Matthias
>> > >
>> > > On 1/11/20 12:26 AM, Vito Jeng wrote:
>> > > > Hi, Matthias & Vinoth,
>> > > >
>> > > > Thanks for the feedback.
>> > > >
>> > > >> What is still unclear to me is, what we gain by having both
>> > > >> `StreamsNotRunningException` and
>> `StateStoreNotAvailableException`. Both
>> > > >> exception are thrown when KafkaStreams is in state
>> PENDING_SHUTDOWN /
>> > > >> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
>> > > >> state store is closed on not -- I can't query it anyway? Maybe I
>> miss
>> > > >> something thought?
>> > > >
>> > > > Yes, both `StreamsNotRunningException` and
>> > > > `StateStoreNotAvailableException` are fatal exception.
>> > > > But `StateStoreNotAvailableException` is fatal exception about state
>> > > store
>> > > > related.
>> > > > I think it would be helpful that if user need to distinguish these
>> two
>> > > > different case to handle it.
>> > > >
>> > > > I'm not very sure, does that make sense?
>> > > >
>> > > >
>> > > > ---
>> > > > Vito
>> > > >
>> > > >
>> > > > On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <vi...@apache.org>
>> > > wrote:
>> > > >
>> > > >> +1 on merging `StreamsNotRunningException` and
>> > > >> `StateStoreNotAvailableException`, both exceptions are fatal
>> anyway. IMO
>> > > >> its best to have these exceptions be about the state store (and not
>> > > streams
>> > > >> state), to easier understanding.
>> > > >>
>> > > >> Additionally, KIP-535 allows for querying of state stores in
>> rebalancing
>> > > >> state. So do we need the StreamsRebalancingException?
>> > > >>
>> > > >>
>> > > >> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io>
>> > > wrote:
>> > > >>> Sorry that I dropped the ball on this...
>> > > >>>
>> > > >>> Thanks for updating the KIP. Overall LGTM now. Feel free to start
>> a
>> > > VOTE
>> > > >>> thread.
>> > > >>>
>> > > >>> What is still unclear to me is, what we gain by having both
>> > > >>> `StreamsNotRunningException` and
>> `StateStoreNotAvailableException`.
>> > > Both
>> > > >>> exception are thrown when KafkaStreams is in state
>> PENDING_SHUTDOWN /
>> > > >>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if
>> the
>> > > >>> state store is closed on not -- I can't query it anyway? Maybe I
>> miss
>> > > >>> something thought?
>> > > >>>
>> > > >>>
>> > > >>> -Matthias
>> > > >>>
>> > > >>>
>> > > >>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>> > > >>>> Sorry for the late reply, thanks for the review.
>> > > >>>>
>> > > >>>>
>> > > >>>>> About `StateStoreMigratedException`:
>> > > >>>>>
>> > > >>>>> Why is it only thrown if the state is REBALANCING? A store
>> might be
>> > > >>>>> migrated during a rebalance, and Kafka Streams might resume
>> back to
>> > > >>>>> RUNNING state and afterward somebody tries to use an old store
>> > > handle.
>> > > >>>>> Also, if state is REBALANCING, should we throw
>> > > >>>>> `StreamThreadRebalancingException`? Hence, I think
>> > > >>>>> `StateStoreMigratedException` does only make sense during
>> `RUNNING`
>> > > >> state.
>> > > >>>>>
>> > > >>>>
>> > > >>>> Thank you point this, already updated.
>> > > >>>>
>> > > >>>>
>> > > >>>> Why do we need to distinguish between
>> > > `KafkaStreamsNotRunningException`
>> > > >>>>> and `StateStoreNotAvailableException`?
>> > > >>>>>
>> > > >>>>
>> > > >>>> `KafkaStreamsNotRunningException` may be caused by various
>> reasons, I
>> > > >> think
>> > > >>>> it would be helpful that the
>> > > >>>> user can distinguish whether it is caused by the state store
>> closed.
>> > > >>>> (Maybe I am wrong...)
>> > > >>>>
>> > > >>>>
>> > > >>>> Last, why do we distinguish between `KafkaStreams` instance and
>> > > >>>>> `StreamsThread`? To me, it seems we should always refer to the
>> > > >> instance,
>> > > >>>>> because that is the level of granularity in which we
>> enable/disable
>> > > >> IQ atm.
>> > > >>>>>
>> > > >>>>
>> > > >>>> Totally agree. Do you mean the naming of state store exceptions?
>> > > >>>> I don't have special reason to distinguish these two.
>> > > >>>> Your suggestion look more reasonable for the exception naming.
>> > > >>>>
>> > > >>>>
>> > > >>>> Last, for `StateStoreMigratedException`, I would add that a user
>> need
>> > > >> to
>> > > >>>>> rediscover the store and cannot blindly retry as the store
>> handle is
>> > > >>>>> invalid and a new store handle must be retrieved. That is a
>> > > difference
>> > > >>>>> to `StreamThreadRebalancingException` that allows for "blind"
>> retries
>> > > >>>>> that either resolve (if the store is still on the same instance
>> after
>> > > >>>>> rebalancing finishes, or changes to
>> `StateStoreMigratedException` if
>> > > >> the
>> > > >>>>> store was migrated away during rebalancing).
>> > > >>>>>
>> > > >>>>
>> > > >>>> Nice, it's great! Thank you.
>> > > >>>>
>> > > >>>>
>> > > >>>> The KIP already updated, please take a look. :)
>> > > >>>>
>> > > >>>>
>> > > >>>>
>> > > >>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
>> > > matthias@confluent.io
>> > > >>>
>> > > >>>> wrote:
>> > > >>>>
>> > > >>>>> Any update on this KIP?
>> > > >>>>>
>> > > >>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>> > > >>>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
>> > > >>>>>>
>> > > >>>>>> About `StateStoreMigratedException`:
>> > > >>>>>>
>> > > >>>>>> Why is it only thrown if the state is REBALANCING? A store
>> might be
>> > > >>>>>> migrated during a rebalance, and Kafka Streams might resume
>> back to
>> > > >>>>>> RUNNING state and afterward somebody tries to use an old store
>> > > >> handle.
>> > > >>>>>> Also, if state is REBALANCING, should we throw
>> > > >>>>>> `StreamThreadRebalancingException`? Hence, I think
>> > > >>>>>> `StateStoreMigratedException` does only make sense during
>> `RUNNING`
>> > > >>>>> state.
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> Why do we need to distinguish between
>> > > >> `KafkaStreamsNotRunningException`
>> > > >>>>>> and `StateStoreNotAvailableException`?
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> Last, why do we distinguish between `KafkaStreams` instance and
>> > > >>>>>> `StreamsThread`? To me, it seems we should always refer to the
>> > > >> instance,
>> > > >>>>>> because that is the level of granularity in which we
>> enable/disable
>> > > >> IQ
>> > > >>>>> atm.
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> Last, for `StateStoreMigratedException`, I would add that a
>> user
>> > > >> need to
>> > > >>>>>> rediscover the store and cannot blindly retry as the store
>> handle is
>> > > >>>>>> invalid and a new store handle must be retrieved. That is a
>> > > >> difference
>> > > >>>>>> to `StreamThreadRebalancingException` that allows for "blind"
>> > > retries
>> > > >>>>>> that either resolve (if the store is still on the same instance
>> > > after
>> > > >>>>>> rebalancing finishes, or changes to
>> `StateStoreMigratedException` if
>> > > >> the
>> > > >>>>>> store was migrated away during rebalancing).
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> -Matthias
>> > > >>>>>>
>> > > >>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>> > > >>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9>
>> > > <https://shorturl.at/CDNT9>
>> > > >> <https://shorturl.at/CDNT9>
>> > > >>>>> <https://shorturl.at/CDNT9>
>> > > >>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>> > > >>>>>>>
>> > > >>>>>>> Please use the following instead: https://shorturl.at/bkKQU
>> > > >>>>>>>
>> > > >>>>>>>
>> > > >>>>>>> ---
>> > > >>>>>>> Vito
>> > > >>>>>>>
>> > > >>>>>>>
>> > > >>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <
>> vito@is-land.com.tw>
>> > > >> wrote:
>> > > >>>>>>>
>> > > >>>>>>>> Thanks, Matthias!
>> > > >>>>>>>>
>> > > >>>>>>>>> About `StreamThreadNotStartedException`:
>> > > >>>>>>>>
>> > > >>>>>>>> Thank you for explanation. I agree with your opinion.
>> > > >>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>> > > >>>>>>>> `StreamThreadNotStartedException`.
>> > > >>>>>>>>
>> > > >>>>>>>> For the case that corresponding thread crashes after we
>> handed out
>> > > >> the
>> > > >>>>>>>> store handle. We may throw `KafkaStreamsNotRunningException`
>> or
>> > > >>>>>>>> `StateStoreMigratedException`.
>> > > >>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>> > > >>>>>>>> `KafkaStreamsNotRunningException` when stream thread is not
>> > > >> running(
>> > > >>>>>>>> https://shorturl.at/CDNT9) or throw
>> `StateStoreMigratedException`
>> > > >> when
>> > > >>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we
>> do not
>> > > >> need
>> > > >>>>> to
>> > > >>>>>>>> add a new type for this case. Does that make sense?
>> > > >>>>>>>>
>> > > >>>>>>>>
>> > > >>>>>>>>> About `KafkaStreamsNotRunningException` vs
>> > > >>>>>>>> `StreamThreadNotRunningException`:
>> > > >>>>>>>>
>> > > >>>>>>>> I understand your point. I rename
>> > > >> `StreamThreadNotRunningException` to
>> > > >>>>>>>> `KafkaStreamsNotRunningException`.
>> > > >>>>>>>>
>> > > >>>>>>>>
>> > > >>>>>>>> About check unknown state store names:
>> > > >>>>>>>> Thank you for the hint. I add a new type
>> > > >> `UnknownStateStoreException`
>> > > >>>>> for
>> > > >>>>>>>> this case.
>> > > >>>>>>>>
>> > > >>>>>>>>
>> > > >>>>>>>>> Also, we should still have fatal exception
>> > > >>>>>>>> `StateStoreNotAvailableException`? Not sure why you remove
>> it?
>> > > >>>>>>>>
>> > > >>>>>>>> Thank you point this, already add it again.
>> > > >>>>>>>>
>> > > >>>>>>>> The KIP already updated, please take a look.
>> > > >>>>>>>>
>> > > >>>>>>>> ---
>> > > >>>>>>>> Vito
>> > > >>>>>>>>
>> > > >>>>>>>
>> > > >>>>>>
>> > > >>>>>
>> > > >>>>>
>> > > >>>>
>> > > >>>
>> > > >>>
>> > > >>
>> > > >
>> > >
>> > >
>> >
>>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Hi, folks,

Thank you suggestion, really appreciate it. :)
I understand your concern. I'll merge StreamsNotRunningException and
StateStoreNotAvailableException.


---
Vito


On Thu, Jan 16, 2020 at 6:22 AM John Roesler <vv...@apache.org> wrote:

> Hey Vito,
>
> Yes, thanks for the KIP. Sorry the discussion has been so long.
> Hopefully, we can close it out soon.
>
> I agree we can drop StreamsNotRunningException in favor of
> just StateStoreNotAvailableException.
>
> Unfortunately, I have some higher-level concerns. The value
> of these exceptions is that they tell you how to handle the
> various situations that can arise while querying a distributed
> data store.
>
> Ideally, as a caller, I should be able to just catch "retriable" or
> "fatal" and handle them appropriately. Otherwise, there's no
> point in having categories, and we should just have all the
> exceptions extend InvalidStateStoreException.
>
> Presently, it's not possible to tell from just the
> "retriable"/"fatal" distinction what to do. You  can tell
> from the descriptions of the various exceptions. E.g.:
>
> Retriable:
>  * StreamsRebalancingException: the exact same call
>     should just be retried until the rebalance is complete
>  * StateStoreMigratedException: the store handle is
>     now invalid, so you need to re-discover the instance
>     and get a new handle on that instance. In other words,
>     the query itself may be valid, but the particular method
>     invocation on this particular instance has encountered
>     a fatal exception.
>
> Fatal:
>  * UnknownStateStoreException: this is truly fatal. No amount
>     of retrying or re-discovering is going to get you a handle on a
>     store that doesn't exist in the cluster.
>  * StateStoreNotAvailableException: this is actually recoverable,
>     since the store might exist in the cluster, but isn't available on
>     this particular instance (which is shut down or whatever).
>
> Personally, I'm not a fan of code bureaucracy, so I'm 100% fine
> with omitting the categorization and just having 5 subclasses
> of InvalidStateStoreException. Each of them would tell you
> how to handle them, and it's not too many to really
> understand and handle each one.
>
> If you really want to have a middle tier, I'd recommend:
> * RetryableStateStoreException: the exact same call
>     should be repeated.
> * RecoverableStateStoreException: the store handle
>     should be discarded and the caller should re-discover
>     the location of the store and repeat the query on the
>     correct instance.
> * FatalStateStoreException: the query/request is totally
>     invalid and will never succeed.
>
> However, attempting to categorize the proposed exceptions
> reveals even problems with this categorization:
> Retriable:
> * StreamsRebalancingException
> Recoverable:
> * StateStoreMigratedException
> * StreamsNotRunningException
> Fatal:
> * UnknownStateStoreException
>
> But StreamsNotStartedException is strange... It means that
> one code path got a handle on a specific KafkaStreams object
> instance and sent it a query before another code path
> invoked the start() method on the exact same object instance.
> It seems like the most likely scenario is that whoever wrote
> the program just forgot to call start() before querying, in
> which case, retrying isn't going to help, and a fatal exception
> is more appropriate. I.e., it sounds like a "first 15 minutes
> experience" problem, and making it fatal would be more
> helpful. Even in a production context, there's no reason not
> to sequence your application startup such that you don't
> accept queries until after Streams is started. Thus, I guess
> I'd categorize it under "fatal".
>
> Regardless of whether you make it fatal or retriable, you'd
> still have a whole category with only one exception in it,
> and the other two categories only have two exceptions.
> Plus, as you pointed out in the KIP, you can't get all
> exceptions in all cases anyway:
> * store() can only throw NotStarted, NotRunning,
>     and Unknown
> * actual store queries can only throw Rebalancing,
>     Migrated, and NotRunning
>
> Thus, in practice also, there are exactly three categories
> and also exactly three exception types. It doesn't seem
> like there's a great advantage to the categories here. To
> avoid the categorization problem and also to clarify what
> exceptions can actually be thrown in different circumstances,
> it seems like we should just:
> * get rid of the middle tier and make all the exceptions
>     extend InvalidStateStoreException
> * drop StateStoreNotAvailableException in favor of
>     StreamsNotRunningException
> * clearly document on all public methods which exceptions
>     need to be handled
>
> How do you feel about this?
> Thanks,
> -John
>
> On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> > Thanks for KIP Vito.
> >
> > Overall the KIP LGTM, but I'd have to agree with others on merging the
> > `StreamsNotRunningException` and `StateStoreNotAvailableException`
> classes.
> >
> > Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> > NOT_RUNNING || ERROR` I'm not even sure how we could distinguish when to
> > use the different
> > exceptions.  Maybe a good middle ground would be to have a detailed
> > exception message.
> >
> > The KIP freeze is close, so I think if we can agree on this, we can wrap
> up
> > the voting soon.
> >
> > Thanks,
> > Bill
> >
> > On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > Vito,
> > >
> > > It's still unclear to me what the advantage is, to have both
> > > `StreamsNotRunningException` and `StateStoreNotAvailableException`?
> > >
> > > For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING / ERROR`
> > > and thus, for a user point of view, why does it matter if the store is
> > > closed on not? I don't understand why/how this information would be
> > > useful? Do you have a concrete example in mind how a user would react
> > > differently to both exceptions?
> > >
> > >
> > > @Vinoth: about `StreamsRebalancingException` -- to me, it seems best to
> > > actually do this on a per-query basis, ie, have an overload
> > > `KafkaStreams#store(...)` that takes a boolean flag that allow to
> > > _disable_ the exception and opt-in to query a active store during
> > > recovery. However, as KIP-535 actually introduces this change in
> > > behavior, I think KIP-216 should not cover this, but KIP-535 should be
> > > updated. I'll follow up on the other KIP thread to raise this point.
> > >
> > >
> > > -Matthias
> > >
> > > On 1/11/20 12:26 AM, Vito Jeng wrote:
> > > > Hi, Matthias & Vinoth,
> > > >
> > > > Thanks for the feedback.
> > > >
> > > >> What is still unclear to me is, what we gain by having both
> > > >> `StreamsNotRunningException` and `StateStoreNotAvailableException`.
> Both
> > > >> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN
> /
> > > >> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> > > >> state store is closed on not -- I can't query it anyway? Maybe I
> miss
> > > >> something thought?
> > > >
> > > > Yes, both `StreamsNotRunningException` and
> > > > `StateStoreNotAvailableException` are fatal exception.
> > > > But `StateStoreNotAvailableException` is fatal exception about state
> > > store
> > > > related.
> > > > I think it would be helpful that if user need to distinguish these
> two
> > > > different case to handle it.
> > > >
> > > > I'm not very sure, does that make sense?
> > > >
> > > >
> > > > ---
> > > > Vito
> > > >
> > > >
> > > > On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <vi...@apache.org>
> > > wrote:
> > > >
> > > >> +1 on merging `StreamsNotRunningException` and
> > > >> `StateStoreNotAvailableException`, both exceptions are fatal
> anyway. IMO
> > > >> its best to have these exceptions be about the state store (and not
> > > streams
> > > >> state), to easier understanding.
> > > >>
> > > >> Additionally, KIP-535 allows for querying of state stores in
> rebalancing
> > > >> state. So do we need the StreamsRebalancingException?
> > > >>
> > > >>
> > > >> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io>
> > > wrote:
> > > >>> Sorry that I dropped the ball on this...
> > > >>>
> > > >>> Thanks for updating the KIP. Overall LGTM now. Feel free to start a
> > > VOTE
> > > >>> thread.
> > > >>>
> > > >>> What is still unclear to me is, what we gain by having both
> > > >>> `StreamsNotRunningException` and `StateStoreNotAvailableException`.
> > > Both
> > > >>> exception are thrown when KafkaStreams is in state
> PENDING_SHUTDOWN /
> > > >>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> > > >>> state store is closed on not -- I can't query it anyway? Maybe I
> miss
> > > >>> something thought?
> > > >>>
> > > >>>
> > > >>> -Matthias
> > > >>>
> > > >>>
> > > >>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> > > >>>> Sorry for the late reply, thanks for the review.
> > > >>>>
> > > >>>>
> > > >>>>> About `StateStoreMigratedException`:
> > > >>>>>
> > > >>>>> Why is it only thrown if the state is REBALANCING? A store might
> be
> > > >>>>> migrated during a rebalance, and Kafka Streams might resume back
> to
> > > >>>>> RUNNING state and afterward somebody tries to use an old store
> > > handle.
> > > >>>>> Also, if state is REBALANCING, should we throw
> > > >>>>> `StreamThreadRebalancingException`? Hence, I think
> > > >>>>> `StateStoreMigratedException` does only make sense during
> `RUNNING`
> > > >> state.
> > > >>>>>
> > > >>>>
> > > >>>> Thank you point this, already updated.
> > > >>>>
> > > >>>>
> > > >>>> Why do we need to distinguish between
> > > `KafkaStreamsNotRunningException`
> > > >>>>> and `StateStoreNotAvailableException`?
> > > >>>>>
> > > >>>>
> > > >>>> `KafkaStreamsNotRunningException` may be caused by various
> reasons, I
> > > >> think
> > > >>>> it would be helpful that the
> > > >>>> user can distinguish whether it is caused by the state store
> closed.
> > > >>>> (Maybe I am wrong...)
> > > >>>>
> > > >>>>
> > > >>>> Last, why do we distinguish between `KafkaStreams` instance and
> > > >>>>> `StreamsThread`? To me, it seems we should always refer to the
> > > >> instance,
> > > >>>>> because that is the level of granularity in which we
> enable/disable
> > > >> IQ atm.
> > > >>>>>
> > > >>>>
> > > >>>> Totally agree. Do you mean the naming of state store exceptions?
> > > >>>> I don't have special reason to distinguish these two.
> > > >>>> Your suggestion look more reasonable for the exception naming.
> > > >>>>
> > > >>>>
> > > >>>> Last, for `StateStoreMigratedException`, I would add that a user
> need
> > > >> to
> > > >>>>> rediscover the store and cannot blindly retry as the store
> handle is
> > > >>>>> invalid and a new store handle must be retrieved. That is a
> > > difference
> > > >>>>> to `StreamThreadRebalancingException` that allows for "blind"
> retries
> > > >>>>> that either resolve (if the store is still on the same instance
> after
> > > >>>>> rebalancing finishes, or changes to
> `StateStoreMigratedException` if
> > > >> the
> > > >>>>> store was migrated away during rebalancing).
> > > >>>>>
> > > >>>>
> > > >>>> Nice, it's great! Thank you.
> > > >>>>
> > > >>>>
> > > >>>> The KIP already updated, please take a look. :)
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> > > matthias@confluent.io
> > > >>>
> > > >>>> wrote:
> > > >>>>
> > > >>>>> Any update on this KIP?
> > > >>>>>
> > > >>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> > > >>>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> > > >>>>>>
> > > >>>>>> About `StateStoreMigratedException`:
> > > >>>>>>
> > > >>>>>> Why is it only thrown if the state is REBALANCING? A store
> might be
> > > >>>>>> migrated during a rebalance, and Kafka Streams might resume
> back to
> > > >>>>>> RUNNING state and afterward somebody tries to use an old store
> > > >> handle.
> > > >>>>>> Also, if state is REBALANCING, should we throw
> > > >>>>>> `StreamThreadRebalancingException`? Hence, I think
> > > >>>>>> `StateStoreMigratedException` does only make sense during
> `RUNNING`
> > > >>>>> state.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Why do we need to distinguish between
> > > >> `KafkaStreamsNotRunningException`
> > > >>>>>> and `StateStoreNotAvailableException`?
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Last, why do we distinguish between `KafkaStreams` instance and
> > > >>>>>> `StreamsThread`? To me, it seems we should always refer to the
> > > >> instance,
> > > >>>>>> because that is the level of granularity in which we
> enable/disable
> > > >> IQ
> > > >>>>> atm.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Last, for `StateStoreMigratedException`, I would add that a user
> > > >> need to
> > > >>>>>> rediscover the store and cannot blindly retry as the store
> handle is
> > > >>>>>> invalid and a new store handle must be retrieved. That is a
> > > >> difference
> > > >>>>>> to `StreamThreadRebalancingException` that allows for "blind"
> > > retries
> > > >>>>>> that either resolve (if the store is still on the same instance
> > > after
> > > >>>>>> rebalancing finishes, or changes to
> `StateStoreMigratedException` if
> > > >> the
> > > >>>>>> store was migrated away during rebalancing).
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> -Matthias
> > > >>>>>>
> > > >>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> > > >>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> > > <https://shorturl.at/CDNT9>
> > > >> <https://shorturl.at/CDNT9>
> > > >>>>> <https://shorturl.at/CDNT9>
> > > >>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> > > >>>>>>>
> > > >>>>>>> Please use the following instead: https://shorturl.at/bkKQU
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> ---
> > > >>>>>>> Vito
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vito@is-land.com.tw
> >
> > > >> wrote:
> > > >>>>>>>
> > > >>>>>>>> Thanks, Matthias!
> > > >>>>>>>>
> > > >>>>>>>>> About `StreamThreadNotStartedException`:
> > > >>>>>>>>
> > > >>>>>>>> Thank you for explanation. I agree with your opinion.
> > > >>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> > > >>>>>>>> `StreamThreadNotStartedException`.
> > > >>>>>>>>
> > > >>>>>>>> For the case that corresponding thread crashes after we
> handed out
> > > >> the
> > > >>>>>>>> store handle. We may throw `KafkaStreamsNotRunningException`
> or
> > > >>>>>>>> `StateStoreMigratedException`.
> > > >>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> > > >>>>>>>> `KafkaStreamsNotRunningException` when stream thread is not
> > > >> running(
> > > >>>>>>>> https://shorturl.at/CDNT9) or throw
> `StateStoreMigratedException`
> > > >> when
> > > >>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we do
> not
> > > >> need
> > > >>>>> to
> > > >>>>>>>> add a new type for this case. Does that make sense?
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>>> About `KafkaStreamsNotRunningException` vs
> > > >>>>>>>> `StreamThreadNotRunningException`:
> > > >>>>>>>>
> > > >>>>>>>> I understand your point. I rename
> > > >> `StreamThreadNotRunningException` to
> > > >>>>>>>> `KafkaStreamsNotRunningException`.
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> About check unknown state store names:
> > > >>>>>>>> Thank you for the hint. I add a new type
> > > >> `UnknownStateStoreException`
> > > >>>>> for
> > > >>>>>>>> this case.
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>>> Also, we should still have fatal exception
> > > >>>>>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
> > > >>>>>>>>
> > > >>>>>>>> Thank you point this, already add it again.
> > > >>>>>>>>
> > > >>>>>>>> The KIP already updated, please take a look.
> > > >>>>>>>>
> > > >>>>>>>> ---
> > > >>>>>>>> Vito
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>
> > > >
> > >
> > >
> >
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by John Roesler <vv...@apache.org>.
Hey Vito,

Yes, thanks for the KIP. Sorry the discussion has been so long. 
Hopefully, we can close it out soon.

I agree we can drop StreamsNotRunningException in favor of 
just StateStoreNotAvailableException.

Unfortunately, I have some higher-level concerns. The value 
of these exceptions is that they tell you how to handle the 
various situations that can arise while querying a distributed 
data store.

Ideally, as a caller, I should be able to just catch "retriable" or 
"fatal" and handle them appropriately. Otherwise, there's no 
point in having categories, and we should just have all the 
exceptions extend InvalidStateStoreException.

Presently, it's not possible to tell from just the 
"retriable"/"fatal" distinction what to do. You  can tell
from the descriptions of the various exceptions. E.g.:

Retriable:
 * StreamsRebalancingException: the exact same call 
    should just be retried until the rebalance is complete
 * StateStoreMigratedException: the store handle is
    now invalid, so you need to re-discover the instance 
    and get a new handle on that instance. In other words,
    the query itself may be valid, but the particular method 
    invocation on this particular instance has encountered 
    a fatal exception.

Fatal:
 * UnknownStateStoreException: this is truly fatal. No amount 
    of retrying or re-discovering is going to get you a handle on a 
    store that doesn't exist in the cluster.
 * StateStoreNotAvailableException: this is actually recoverable, 
    since the store might exist in the cluster, but isn't available on 
    this particular instance (which is shut down or whatever).

Personally, I'm not a fan of code bureaucracy, so I'm 100% fine 
with omitting the categorization and just having 5 subclasses 
of InvalidStateStoreException. Each of them would tell you 
how to handle them, and it's not too many to really 
understand and handle each one.

If you really want to have a middle tier, I'd recommend:
* RetryableStateStoreException: the exact same call 
    should be repeated.
* RecoverableStateStoreException: the store handle 
    should be discarded and the caller should re-discover 
    the location of the store and repeat the query on the 
    correct instance.
* FatalStateStoreException: the query/request is totally 
    invalid and will never succeed.

However, attempting to categorize the proposed exceptions 
reveals even problems with this categorization:
Retriable:
* StreamsRebalancingException
Recoverable:
* StateStoreMigratedException
* StreamsNotRunningException
Fatal:
* UnknownStateStoreException

But StreamsNotStartedException is strange... It means that 
one code path got a handle on a specific KafkaStreams object 
instance and sent it a query before another code path 
invoked the start() method on the exact same object instance. 
It seems like the most likely scenario is that whoever wrote 
the program just forgot to call start() before querying, in 
which case, retrying isn't going to help, and a fatal exception 
is more appropriate. I.e., it sounds like a "first 15 minutes 
experience" problem, and making it fatal would be more 
helpful. Even in a production context, there's no reason not 
to sequence your application startup such that you don't 
accept queries until after Streams is started. Thus, I guess 
I'd categorize it under "fatal".

Regardless of whether you make it fatal or retriable, you'd 
still have a whole category with only one exception in it, 
and the other two categories only have two exceptions. 
Plus, as you pointed out in the KIP, you can't get all 
exceptions in all cases anyway:
* store() can only throw NotStarted, NotRunning, 
    and Unknown
* actual store queries can only throw Rebalancing, 
    Migrated, and NotRunning

Thus, in practice also, there are exactly three categories 
and also exactly three exception types. It doesn't seem 
like there's a great advantage to the categories here. To 
avoid the categorization problem and also to clarify what 
exceptions can actually be thrown in different circumstances, 
it seems like we should just:
* get rid of the middle tier and make all the exceptions 
    extend InvalidStateStoreException
* drop StateStoreNotAvailableException in favor of 
    StreamsNotRunningException
* clearly document on all public methods which exceptions 
    need to be handled

How do you feel about this?
Thanks,
-John

On Wed, Jan 15, 2020, at 15:13, Bill Bejeck wrote:
> Thanks for KIP Vito.
> 
> Overall the KIP LGTM, but I'd have to agree with others on merging the
> `StreamsNotRunningException` and `StateStoreNotAvailableException` classes.
> 
> Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
> NOT_RUNNING || ERROR` I'm not even sure how we could distinguish when to
> use the different
> exceptions.  Maybe a good middle ground would be to have a detailed
> exception message.
> 
> The KIP freeze is close, so I think if we can agree on this, we can wrap up
> the voting soon.
> 
> Thanks,
> Bill
> 
> On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
> > Vito,
> >
> > It's still unclear to me what the advantage is, to have both
> > `StreamsNotRunningException` and `StateStoreNotAvailableException`?
> >
> > For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING / ERROR`
> > and thus, for a user point of view, why does it matter if the store is
> > closed on not? I don't understand why/how this information would be
> > useful? Do you have a concrete example in mind how a user would react
> > differently to both exceptions?
> >
> >
> > @Vinoth: about `StreamsRebalancingException` -- to me, it seems best to
> > actually do this on a per-query basis, ie, have an overload
> > `KafkaStreams#store(...)` that takes a boolean flag that allow to
> > _disable_ the exception and opt-in to query a active store during
> > recovery. However, as KIP-535 actually introduces this change in
> > behavior, I think KIP-216 should not cover this, but KIP-535 should be
> > updated. I'll follow up on the other KIP thread to raise this point.
> >
> >
> > -Matthias
> >
> > On 1/11/20 12:26 AM, Vito Jeng wrote:
> > > Hi, Matthias & Vinoth,
> > >
> > > Thanks for the feedback.
> > >
> > >> What is still unclear to me is, what we gain by having both
> > >> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
> > >> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> > >> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> > >> state store is closed on not -- I can't query it anyway? Maybe I miss
> > >> something thought?
> > >
> > > Yes, both `StreamsNotRunningException` and
> > > `StateStoreNotAvailableException` are fatal exception.
> > > But `StateStoreNotAvailableException` is fatal exception about state
> > store
> > > related.
> > > I think it would be helpful that if user need to distinguish these two
> > > different case to handle it.
> > >
> > > I'm not very sure, does that make sense?
> > >
> > >
> > > ---
> > > Vito
> > >
> > >
> > > On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <vi...@apache.org>
> > wrote:
> > >
> > >> +1 on merging `StreamsNotRunningException` and
> > >> `StateStoreNotAvailableException`, both exceptions are fatal anyway. IMO
> > >> its best to have these exceptions be about the state store (and not
> > streams
> > >> state), to easier understanding.
> > >>
> > >> Additionally, KIP-535 allows for querying of state stores in rebalancing
> > >> state. So do we need the StreamsRebalancingException?
> > >>
> > >>
> > >> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io>
> > wrote:
> > >>> Sorry that I dropped the ball on this...
> > >>>
> > >>> Thanks for updating the KIP. Overall LGTM now. Feel free to start a
> > VOTE
> > >>> thread.
> > >>>
> > >>> What is still unclear to me is, what we gain by having both
> > >>> `StreamsNotRunningException` and `StateStoreNotAvailableException`.
> > Both
> > >>> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> > >>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> > >>> state store is closed on not -- I can't query it anyway? Maybe I miss
> > >>> something thought?
> > >>>
> > >>>
> > >>> -Matthias
> > >>>
> > >>>
> > >>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> > >>>> Sorry for the late reply, thanks for the review.
> > >>>>
> > >>>>
> > >>>>> About `StateStoreMigratedException`:
> > >>>>>
> > >>>>> Why is it only thrown if the state is REBALANCING? A store might be
> > >>>>> migrated during a rebalance, and Kafka Streams might resume back to
> > >>>>> RUNNING state and afterward somebody tries to use an old store
> > handle.
> > >>>>> Also, if state is REBALANCING, should we throw
> > >>>>> `StreamThreadRebalancingException`? Hence, I think
> > >>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
> > >> state.
> > >>>>>
> > >>>>
> > >>>> Thank you point this, already updated.
> > >>>>
> > >>>>
> > >>>> Why do we need to distinguish between
> > `KafkaStreamsNotRunningException`
> > >>>>> and `StateStoreNotAvailableException`?
> > >>>>>
> > >>>>
> > >>>> `KafkaStreamsNotRunningException` may be caused by various reasons, I
> > >> think
> > >>>> it would be helpful that the
> > >>>> user can distinguish whether it is caused by the state store closed.
> > >>>> (Maybe I am wrong...)
> > >>>>
> > >>>>
> > >>>> Last, why do we distinguish between `KafkaStreams` instance and
> > >>>>> `StreamsThread`? To me, it seems we should always refer to the
> > >> instance,
> > >>>>> because that is the level of granularity in which we enable/disable
> > >> IQ atm.
> > >>>>>
> > >>>>
> > >>>> Totally agree. Do you mean the naming of state store exceptions?
> > >>>> I don't have special reason to distinguish these two.
> > >>>> Your suggestion look more reasonable for the exception naming.
> > >>>>
> > >>>>
> > >>>> Last, for `StateStoreMigratedException`, I would add that a user need
> > >> to
> > >>>>> rediscover the store and cannot blindly retry as the store handle is
> > >>>>> invalid and a new store handle must be retrieved. That is a
> > difference
> > >>>>> to `StreamThreadRebalancingException` that allows for "blind" retries
> > >>>>> that either resolve (if the store is still on the same instance after
> > >>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
> > >> the
> > >>>>> store was migrated away during rebalancing).
> > >>>>>
> > >>>>
> > >>>> Nice, it's great! Thank you.
> > >>>>
> > >>>>
> > >>>> The KIP already updated, please take a look. :)
> > >>>>
> > >>>>
> > >>>>
> > >>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> > matthias@confluent.io
> > >>>
> > >>>> wrote:
> > >>>>
> > >>>>> Any update on this KIP?
> > >>>>>
> > >>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> > >>>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> > >>>>>>
> > >>>>>> About `StateStoreMigratedException`:
> > >>>>>>
> > >>>>>> Why is it only thrown if the state is REBALANCING? A store might be
> > >>>>>> migrated during a rebalance, and Kafka Streams might resume back to
> > >>>>>> RUNNING state and afterward somebody tries to use an old store
> > >> handle.
> > >>>>>> Also, if state is REBALANCING, should we throw
> > >>>>>> `StreamThreadRebalancingException`? Hence, I think
> > >>>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
> > >>>>> state.
> > >>>>>>
> > >>>>>>
> > >>>>>> Why do we need to distinguish between
> > >> `KafkaStreamsNotRunningException`
> > >>>>>> and `StateStoreNotAvailableException`?
> > >>>>>>
> > >>>>>>
> > >>>>>> Last, why do we distinguish between `KafkaStreams` instance and
> > >>>>>> `StreamsThread`? To me, it seems we should always refer to the
> > >> instance,
> > >>>>>> because that is the level of granularity in which we enable/disable
> > >> IQ
> > >>>>> atm.
> > >>>>>>
> > >>>>>>
> > >>>>>> Last, for `StateStoreMigratedException`, I would add that a user
> > >> need to
> > >>>>>> rediscover the store and cannot blindly retry as the store handle is
> > >>>>>> invalid and a new store handle must be retrieved. That is a
> > >> difference
> > >>>>>> to `StreamThreadRebalancingException` that allows for "blind"
> > retries
> > >>>>>> that either resolve (if the store is still on the same instance
> > after
> > >>>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
> > >> the
> > >>>>>> store was migrated away during rebalancing).
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> -Matthias
> > >>>>>>
> > >>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> > >>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> > <https://shorturl.at/CDNT9>
> > >> <https://shorturl.at/CDNT9>
> > >>>>> <https://shorturl.at/CDNT9>
> > >>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> > >>>>>>>
> > >>>>>>> Please use the following instead: https://shorturl.at/bkKQU
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> ---
> > >>>>>>> Vito
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw>
> > >> wrote:
> > >>>>>>>
> > >>>>>>>> Thanks, Matthias!
> > >>>>>>>>
> > >>>>>>>>> About `StreamThreadNotStartedException`:
> > >>>>>>>>
> > >>>>>>>> Thank you for explanation. I agree with your opinion.
> > >>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> > >>>>>>>> `StreamThreadNotStartedException`.
> > >>>>>>>>
> > >>>>>>>> For the case that corresponding thread crashes after we handed out
> > >> the
> > >>>>>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
> > >>>>>>>> `StateStoreMigratedException`.
> > >>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> > >>>>>>>> `KafkaStreamsNotRunningException` when stream thread is not
> > >> running(
> > >>>>>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException`
> > >> when
> > >>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not
> > >> need
> > >>>>> to
> > >>>>>>>> add a new type for this case. Does that make sense?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>> About `KafkaStreamsNotRunningException` vs
> > >>>>>>>> `StreamThreadNotRunningException`:
> > >>>>>>>>
> > >>>>>>>> I understand your point. I rename
> > >> `StreamThreadNotRunningException` to
> > >>>>>>>> `KafkaStreamsNotRunningException`.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> About check unknown state store names:
> > >>>>>>>> Thank you for the hint. I add a new type
> > >> `UnknownStateStoreException`
> > >>>>> for
> > >>>>>>>> this case.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>> Also, we should still have fatal exception
> > >>>>>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
> > >>>>>>>>
> > >>>>>>>> Thank you point this, already add it again.
> > >>>>>>>>
> > >>>>>>>> The KIP already updated, please take a look.
> > >>>>>>>>
> > >>>>>>>> ---
> > >>>>>>>> Vito
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>>
> > >>
> > >
> >
> >
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Bill Bejeck <bb...@gmail.com>.
Thanks for KIP Vito.

Overall the KIP LGTM, but I'd have to agree with others on merging the
`StreamsNotRunningException` and `StateStoreNotAvailableException` classes.

Since in both cases, the thread state is in `PENDING_SHUTDOWN ||
NOT_RUNNING || ERROR` I'm not even sure how we could distinguish when to
use the different
exceptions.  Maybe a good middle ground would be to have a detailed
exception message.

The KIP freeze is close, so I think if we can agree on this, we can wrap up
the voting soon.

Thanks,
Bill

On Tue, Jan 14, 2020 at 2:12 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Vito,
>
> It's still unclear to me what the advantage is, to have both
> `StreamsNotRunningException` and `StateStoreNotAvailableException`?
>
> For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING / ERROR`
> and thus, for a user point of view, why does it matter if the store is
> closed on not? I don't understand why/how this information would be
> useful? Do you have a concrete example in mind how a user would react
> differently to both exceptions?
>
>
> @Vinoth: about `StreamsRebalancingException` -- to me, it seems best to
> actually do this on a per-query basis, ie, have an overload
> `KafkaStreams#store(...)` that takes a boolean flag that allow to
> _disable_ the exception and opt-in to query a active store during
> recovery. However, as KIP-535 actually introduces this change in
> behavior, I think KIP-216 should not cover this, but KIP-535 should be
> updated. I'll follow up on the other KIP thread to raise this point.
>
>
> -Matthias
>
> On 1/11/20 12:26 AM, Vito Jeng wrote:
> > Hi, Matthias & Vinoth,
> >
> > Thanks for the feedback.
> >
> >> What is still unclear to me is, what we gain by having both
> >> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
> >> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> >> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> >> state store is closed on not -- I can't query it anyway? Maybe I miss
> >> something thought?
> >
> > Yes, both `StreamsNotRunningException` and
> > `StateStoreNotAvailableException` are fatal exception.
> > But `StateStoreNotAvailableException` is fatal exception about state
> store
> > related.
> > I think it would be helpful that if user need to distinguish these two
> > different case to handle it.
> >
> > I'm not very sure, does that make sense?
> >
> >
> > ---
> > Vito
> >
> >
> > On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <vi...@apache.org>
> wrote:
> >
> >> +1 on merging `StreamsNotRunningException` and
> >> `StateStoreNotAvailableException`, both exceptions are fatal anyway. IMO
> >> its best to have these exceptions be about the state store (and not
> streams
> >> state), to easier understanding.
> >>
> >> Additionally, KIP-535 allows for querying of state stores in rebalancing
> >> state. So do we need the StreamsRebalancingException?
> >>
> >>
> >> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io>
> wrote:
> >>> Sorry that I dropped the ball on this...
> >>>
> >>> Thanks for updating the KIP. Overall LGTM now. Feel free to start a
> VOTE
> >>> thread.
> >>>
> >>> What is still unclear to me is, what we gain by having both
> >>> `StreamsNotRunningException` and `StateStoreNotAvailableException`.
> Both
> >>> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> >>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> >>> state store is closed on not -- I can't query it anyway? Maybe I miss
> >>> something thought?
> >>>
> >>>
> >>> -Matthias
> >>>
> >>>
> >>> On 11/3/19 6:07 PM, Vito Jeng wrote:
> >>>> Sorry for the late reply, thanks for the review.
> >>>>
> >>>>
> >>>>> About `StateStoreMigratedException`:
> >>>>>
> >>>>> Why is it only thrown if the state is REBALANCING? A store might be
> >>>>> migrated during a rebalance, and Kafka Streams might resume back to
> >>>>> RUNNING state and afterward somebody tries to use an old store
> handle.
> >>>>> Also, if state is REBALANCING, should we throw
> >>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
> >> state.
> >>>>>
> >>>>
> >>>> Thank you point this, already updated.
> >>>>
> >>>>
> >>>> Why do we need to distinguish between
> `KafkaStreamsNotRunningException`
> >>>>> and `StateStoreNotAvailableException`?
> >>>>>
> >>>>
> >>>> `KafkaStreamsNotRunningException` may be caused by various reasons, I
> >> think
> >>>> it would be helpful that the
> >>>> user can distinguish whether it is caused by the state store closed.
> >>>> (Maybe I am wrong...)
> >>>>
> >>>>
> >>>> Last, why do we distinguish between `KafkaStreams` instance and
> >>>>> `StreamsThread`? To me, it seems we should always refer to the
> >> instance,
> >>>>> because that is the level of granularity in which we enable/disable
> >> IQ atm.
> >>>>>
> >>>>
> >>>> Totally agree. Do you mean the naming of state store exceptions?
> >>>> I don't have special reason to distinguish these two.
> >>>> Your suggestion look more reasonable for the exception naming.
> >>>>
> >>>>
> >>>> Last, for `StateStoreMigratedException`, I would add that a user need
> >> to
> >>>>> rediscover the store and cannot blindly retry as the store handle is
> >>>>> invalid and a new store handle must be retrieved. That is a
> difference
> >>>>> to `StreamThreadRebalancingException` that allows for "blind" retries
> >>>>> that either resolve (if the store is still on the same instance after
> >>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
> >> the
> >>>>> store was migrated away during rebalancing).
> >>>>>
> >>>>
> >>>> Nice, it's great! Thank you.
> >>>>
> >>>>
> >>>> The KIP already updated, please take a look. :)
> >>>>
> >>>>
> >>>>
> >>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <
> matthias@confluent.io
> >>>
> >>>> wrote:
> >>>>
> >>>>> Any update on this KIP?
> >>>>>
> >>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >>>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> >>>>>>
> >>>>>> About `StateStoreMigratedException`:
> >>>>>>
> >>>>>> Why is it only thrown if the state is REBALANCING? A store might be
> >>>>>> migrated during a rebalance, and Kafka Streams might resume back to
> >>>>>> RUNNING state and afterward somebody tries to use an old store
> >> handle.
> >>>>>> Also, if state is REBALANCING, should we throw
> >>>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
> >>>>> state.
> >>>>>>
> >>>>>>
> >>>>>> Why do we need to distinguish between
> >> `KafkaStreamsNotRunningException`
> >>>>>> and `StateStoreNotAvailableException`?
> >>>>>>
> >>>>>>
> >>>>>> Last, why do we distinguish between `KafkaStreams` instance and
> >>>>>> `StreamsThread`? To me, it seems we should always refer to the
> >> instance,
> >>>>>> because that is the level of granularity in which we enable/disable
> >> IQ
> >>>>> atm.
> >>>>>>
> >>>>>>
> >>>>>> Last, for `StateStoreMigratedException`, I would add that a user
> >> need to
> >>>>>> rediscover the store and cannot blindly retry as the store handle is
> >>>>>> invalid and a new store handle must be retrieved. That is a
> >> difference
> >>>>>> to `StreamThreadRebalancingException` that allows for "blind"
> retries
> >>>>>> that either resolve (if the store is still on the same instance
> after
> >>>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
> >> the
> >>>>>> store was migrated away during rebalancing).
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >> <https://shorturl.at/CDNT9>
> >>>>> <https://shorturl.at/CDNT9>
> >>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>>
> >>>>>>> Please use the following instead: https://shorturl.at/bkKQU
> >>>>>>>
> >>>>>>>
> >>>>>>> ---
> >>>>>>> Vito
> >>>>>>>
> >>>>>>>
> >>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw>
> >> wrote:
> >>>>>>>
> >>>>>>>> Thanks, Matthias!
> >>>>>>>>
> >>>>>>>>> About `StreamThreadNotStartedException`:
> >>>>>>>>
> >>>>>>>> Thank you for explanation. I agree with your opinion.
> >>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >>>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>>
> >>>>>>>> For the case that corresponding thread crashes after we handed out
> >> the
> >>>>>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
> >>>>>>>> `StateStoreMigratedException`.
> >>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>>> `KafkaStreamsNotRunningException` when stream thread is not
> >> running(
> >>>>>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException`
> >> when
> >>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not
> >> need
> >>>>> to
> >>>>>>>> add a new type for this case. Does that make sense?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>> About `KafkaStreamsNotRunningException` vs
> >>>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>>
> >>>>>>>> I understand your point. I rename
> >> `StreamThreadNotRunningException` to
> >>>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> About check unknown state store names:
> >>>>>>>> Thank you for the hint. I add a new type
> >> `UnknownStateStoreException`
> >>>>> for
> >>>>>>>> this case.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>> Also, we should still have fatal exception
> >>>>>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
> >>>>>>>>
> >>>>>>>> Thank you point this, already add it again.
> >>>>>>>>
> >>>>>>>> The KIP already updated, please take a look.
> >>>>>>>>
> >>>>>>>> ---
> >>>>>>>> Vito
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Vito,

It's still unclear to me what the advantage is, to have both
`StreamsNotRunningException` and `StateStoreNotAvailableException`?

For both cased, the state is `PENDING_SHUTDOWN / NOT_RUNNING / ERROR`
and thus, for a user point of view, why does it matter if the store is
closed on not? I don't understand why/how this information would be
useful? Do you have a concrete example in mind how a user would react
differently to both exceptions?


@Vinoth: about `StreamsRebalancingException` -- to me, it seems best to
actually do this on a per-query basis, ie, have an overload
`KafkaStreams#store(...)` that takes a boolean flag that allow to
_disable_ the exception and opt-in to query a active store during
recovery. However, as KIP-535 actually introduces this change in
behavior, I think KIP-216 should not cover this, but KIP-535 should be
updated. I'll follow up on the other KIP thread to raise this point.


-Matthias

On 1/11/20 12:26 AM, Vito Jeng wrote:
> Hi, Matthias & Vinoth,
> 
> Thanks for the feedback.
> 
>> What is still unclear to me is, what we gain by having both
>> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
>> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
>> state store is closed on not -- I can't query it anyway? Maybe I miss
>> something thought?
> 
> Yes, both `StreamsNotRunningException` and
> `StateStoreNotAvailableException` are fatal exception.
> But `StateStoreNotAvailableException` is fatal exception about state store
> related.
> I think it would be helpful that if user need to distinguish these two
> different case to handle it.
> 
> I'm not very sure, does that make sense?
> 
> 
> ---
> Vito
> 
> 
> On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <vi...@apache.org> wrote:
> 
>> +1 on merging `StreamsNotRunningException` and
>> `StateStoreNotAvailableException`, both exceptions are fatal anyway. IMO
>> its best to have these exceptions be about the state store (and not streams
>> state), to easier understanding.
>>
>> Additionally, KIP-535 allows for querying of state stores in rebalancing
>> state. So do we need the StreamsRebalancingException?
>>
>>
>> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io> wrote:
>>> Sorry that I dropped the ball on this...
>>>
>>> Thanks for updating the KIP. Overall LGTM now. Feel free to start a VOTE
>>> thread.
>>>
>>> What is still unclear to me is, what we gain by having both
>>> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
>>> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
>>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
>>> state store is closed on not -- I can't query it anyway? Maybe I miss
>>> something thought?
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>>>> Sorry for the late reply, thanks for the review.
>>>>
>>>>
>>>>> About `StateStoreMigratedException`:
>>>>>
>>>>> Why is it only thrown if the state is REBALANCING? A store might be
>>>>> migrated during a rebalance, and Kafka Streams might resume back to
>>>>> RUNNING state and afterward somebody tries to use an old store handle.
>>>>> Also, if state is REBALANCING, should we throw
>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
>> state.
>>>>>
>>>>
>>>> Thank you point this, already updated.
>>>>
>>>>
>>>> Why do we need to distinguish between `KafkaStreamsNotRunningException`
>>>>> and `StateStoreNotAvailableException`?
>>>>>
>>>>
>>>> `KafkaStreamsNotRunningException` may be caused by various reasons, I
>> think
>>>> it would be helpful that the
>>>> user can distinguish whether it is caused by the state store closed.
>>>> (Maybe I am wrong...)
>>>>
>>>>
>>>> Last, why do we distinguish between `KafkaStreams` instance and
>>>>> `StreamsThread`? To me, it seems we should always refer to the
>> instance,
>>>>> because that is the level of granularity in which we enable/disable
>> IQ atm.
>>>>>
>>>>
>>>> Totally agree. Do you mean the naming of state store exceptions?
>>>> I don't have special reason to distinguish these two.
>>>> Your suggestion look more reasonable for the exception naming.
>>>>
>>>>
>>>> Last, for `StateStoreMigratedException`, I would add that a user need
>> to
>>>>> rediscover the store and cannot blindly retry as the store handle is
>>>>> invalid and a new store handle must be retrieved. That is a difference
>>>>> to `StreamThreadRebalancingException` that allows for "blind" retries
>>>>> that either resolve (if the store is still on the same instance after
>>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
>> the
>>>>> store was migrated away during rebalancing).
>>>>>
>>>>
>>>> Nice, it's great! Thank you.
>>>>
>>>>
>>>> The KIP already updated, please take a look. :)
>>>>
>>>>
>>>>
>>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <matthias@confluent.io
>>>
>>>> wrote:
>>>>
>>>>> Any update on this KIP?
>>>>>
>>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>>>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
>>>>>>
>>>>>> About `StateStoreMigratedException`:
>>>>>>
>>>>>> Why is it only thrown if the state is REBALANCING? A store might be
>>>>>> migrated during a rebalance, and Kafka Streams might resume back to
>>>>>> RUNNING state and afterward somebody tries to use an old store
>> handle.
>>>>>> Also, if state is REBALANCING, should we throw
>>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
>>>>> state.
>>>>>>
>>>>>>
>>>>>> Why do we need to distinguish between
>> `KafkaStreamsNotRunningException`
>>>>>> and `StateStoreNotAvailableException`?
>>>>>>
>>>>>>
>>>>>> Last, why do we distinguish between `KafkaStreams` instance and
>>>>>> `StreamsThread`? To me, it seems we should always refer to the
>> instance,
>>>>>> because that is the level of granularity in which we enable/disable
>> IQ
>>>>> atm.
>>>>>>
>>>>>>
>>>>>> Last, for `StateStoreMigratedException`, I would add that a user
>> need to
>>>>>> rediscover the store and cannot blindly retry as the store handle is
>>>>>> invalid and a new store handle must be retrieved. That is a
>> difference
>>>>>> to `StreamThreadRebalancingException` that allows for "blind" retries
>>>>>> that either resolve (if the store is still on the same instance after
>>>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
>> the
>>>>>> store was migrated away during rebalancing).
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9>
>>>>> <https://shorturl.at/CDNT9>
>>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>>>>>>>
>>>>>>> Please use the following instead: https://shorturl.at/bkKQU
>>>>>>>
>>>>>>>
>>>>>>> ---
>>>>>>> Vito
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw>
>> wrote:
>>>>>>>
>>>>>>>> Thanks, Matthias!
>>>>>>>>
>>>>>>>>> About `StreamThreadNotStartedException`:
>>>>>>>>
>>>>>>>> Thank you for explanation. I agree with your opinion.
>>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>>
>>>>>>>> For the case that corresponding thread crashes after we handed out
>> the
>>>>>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
>>>>>>>> `StateStoreMigratedException`.
>>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>>>>>>>> `KafkaStreamsNotRunningException` when stream thread is not
>> running(
>>>>>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException`
>> when
>>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not
>> need
>>>>> to
>>>>>>>> add a new type for this case. Does that make sense?
>>>>>>>>
>>>>>>>>
>>>>>>>>> About `KafkaStreamsNotRunningException` vs
>>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>>
>>>>>>>> I understand your point. I rename
>> `StreamThreadNotRunningException` to
>>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>>
>>>>>>>>
>>>>>>>> About check unknown state store names:
>>>>>>>> Thank you for the hint. I add a new type
>> `UnknownStateStoreException`
>>>>> for
>>>>>>>> this case.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Also, we should still have fatal exception
>>>>>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
>>>>>>>>
>>>>>>>> Thank you point this, already add it again.
>>>>>>>>
>>>>>>>> The KIP already updated, please take a look.
>>>>>>>>
>>>>>>>> ---
>>>>>>>> Vito
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>>
>>
> 


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vinoth Chandar <vc...@confluent.io>.
+1 on a streams config to control this behavior. I have no strong opinions
on the default, but I would pick allowing to query if standbys are enabled
else throw the exception..
But we can keep it simpler, throw exception by default and have a flag to
turn it off, as you suggest as well.

On Thu, Jan 9, 2020 at 12:01 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Good question about `StreamsRebalancingException` -- when this KIP was
> started, KIP-535 was not on the horizon yet.
>
> What I am wondering is, if we should allow people to opt-in into
> querying during a rebalance, or to be more precise during a restore (if
> a state store is not migrated, it will be up-to-date during a rebalance
> and can be queried returning correct, ie, non-stall, data)?
>
> Otherwise, if people want to get only correct results, ie, they never
> want to query stall state, they have no way to implement it, because
> they are always subject to a race condition.
>
> For this case, we could have a `StateStoreIsRecoveringException` (or
> similar) that is only throw during a restore phases (and people can
> opt-in / opt-out if this exception should be throws or not, ie, if they
> want to query stall state during recovery or not).
>
> It's unclear to me though atm, how a user would opt-in/opt-out and what
> the default should be (maybe better to throw the exception by default to
> have strong consistency guarantees by default?)
>
>
> -Matthias
>
>
> On 1/9/20 11:35 AM, Vinoth Chandar wrote:
> > +1 on merging `StreamsNotRunningException` and
> `StateStoreNotAvailableException`, both exceptions are fatal anyway. IMO
> its best to have these exceptions be about the state store (and not streams
> state), to easier understanding.
> >
> > Additionally, KIP-535 allows for querying of state stores in rebalancing
> state. So do we need the StreamsRebalancingException?
> >
> >
> > On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io>
> wrote:
> >> Sorry that I dropped the ball on this...
> >>
> >> Thanks for updating the KIP. Overall LGTM now. Feel free to start a VOTE
> >> thread.
> >>
> >> What is still unclear to me is, what we gain by having both
> >> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
> >> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> >> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> >> state store is closed on not -- I can't query it anyway? Maybe I miss
> >> something thought?
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 11/3/19 6:07 PM, Vito Jeng wrote:
> >>> Sorry for the late reply, thanks for the review.
> >>>
> >>>
> >>>> About `StateStoreMigratedException`:
> >>>>
> >>>> Why is it only thrown if the state is REBALANCING? A store might be
> >>>> migrated during a rebalance, and Kafka Streams might resume back to
> >>>> RUNNING state and afterward somebody tries to use an old store handle.
> >>>> Also, if state is REBALANCING, should we throw
> >>>> `StreamThreadRebalancingException`? Hence, I think
> >>>> `StateStoreMigratedException` does only make sense during `RUNNING`
> state.
> >>>>
> >>>
> >>> Thank you point this, already updated.
> >>>
> >>>
> >>> Why do we need to distinguish between `KafkaStreamsNotRunningException`
> >>>> and `StateStoreNotAvailableException`?
> >>>>
> >>>
> >>> `KafkaStreamsNotRunningException` may be caused by various reasons, I
> think
> >>> it would be helpful that the
> >>> user can distinguish whether it is caused by the state store closed.
> >>> (Maybe I am wrong...)
> >>>
> >>>
> >>> Last, why do we distinguish between `KafkaStreams` instance and
> >>>> `StreamsThread`? To me, it seems we should always refer to the
> instance,
> >>>> because that is the level of granularity in which we enable/disable
> IQ atm.
> >>>>
> >>>
> >>> Totally agree. Do you mean the naming of state store exceptions?
> >>> I don't have special reason to distinguish these two.
> >>> Your suggestion look more reasonable for the exception naming.
> >>>
> >>>
> >>> Last, for `StateStoreMigratedException`, I would add that a user need
> to
> >>>> rediscover the store and cannot blindly retry as the store handle is
> >>>> invalid and a new store handle must be retrieved. That is a difference
> >>>> to `StreamThreadRebalancingException` that allows for "blind" retries
> >>>> that either resolve (if the store is still on the same instance after
> >>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
> the
> >>>> store was migrated away during rebalancing).
> >>>>
> >>>
> >>> Nice, it's great! Thank you.
> >>>
> >>>
> >>> The KIP already updated, please take a look. :)
> >>>
> >>>
> >>>
> >>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <matthias@confluent.io
> >
> >>> wrote:
> >>>
> >>>> Any update on this KIP?
> >>>>
> >>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> >>>>>
> >>>>> About `StateStoreMigratedException`:
> >>>>>
> >>>>> Why is it only thrown if the state is REBALANCING? A store might be
> >>>>> migrated during a rebalance, and Kafka Streams might resume back to
> >>>>> RUNNING state and afterward somebody tries to use an old store
> handle.
> >>>>> Also, if state is REBALANCING, should we throw
> >>>>> `StreamThreadRebalancingException`? Hence, I think
> >>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
> >>>> state.
> >>>>>
> >>>>>
> >>>>> Why do we need to distinguish between
> `KafkaStreamsNotRunningException`
> >>>>> and `StateStoreNotAvailableException`?
> >>>>>
> >>>>>
> >>>>> Last, why do we distinguish between `KafkaStreams` instance and
> >>>>> `StreamsThread`? To me, it seems we should always refer to the
> instance,
> >>>>> because that is the level of granularity in which we enable/disable
> IQ
> >>>> atm.
> >>>>>
> >>>>>
> >>>>> Last, for `StateStoreMigratedException`, I would add that a user
> need to
> >>>>> rediscover the store and cannot blindly retry as the store handle is
> >>>>> invalid and a new store handle must be retrieved. That is a
> difference
> >>>>> to `StreamThreadRebalancingException` that allows for "blind" retries
> >>>>> that either resolve (if the store is still on the same instance after
> >>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if
> the
> >>>>> store was migrated away during rebalancing).
> >>>>>
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >>>>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >>>> <https://shorturl.at/CDNT9>
> >>>>>> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>>>
> >>>>>> Please use the following instead: https://shorturl.at/bkKQU
> >>>>>>
> >>>>>>
> >>>>>> ---
> >>>>>> Vito
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw>
> wrote:
> >>>>>>
> >>>>>>> Thanks, Matthias!
> >>>>>>>
> >>>>>>>> About `StreamThreadNotStartedException`:
> >>>>>>>
> >>>>>>> Thank you for explanation. I agree with your opinion.
> >>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >>>>>>> `StreamThreadNotStartedException`.
> >>>>>>>
> >>>>>>> For the case that corresponding thread crashes after we handed out
> the
> >>>>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
> >>>>>>> `StateStoreMigratedException`.
> >>>>>>> In `StreamThreadStateStoreProvider`, we would throw
> >>>>>>> `KafkaStreamsNotRunningException` when stream thread is not
> running(
> >>>>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException`
> when
> >>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not
> need
> >>>> to
> >>>>>>> add a new type for this case. Does that make sense?
> >>>>>>>
> >>>>>>>
> >>>>>>>> About `KafkaStreamsNotRunningException` vs
> >>>>>>> `StreamThreadNotRunningException`:
> >>>>>>>
> >>>>>>> I understand your point. I rename
> `StreamThreadNotRunningException` to
> >>>>>>> `KafkaStreamsNotRunningException`.
> >>>>>>>
> >>>>>>>
> >>>>>>> About check unknown state store names:
> >>>>>>> Thank you for the hint. I add a new type
> `UnknownStateStoreException`
> >>>> for
> >>>>>>> this case.
> >>>>>>>
> >>>>>>>
> >>>>>>>> Also, we should still have fatal exception
> >>>>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
> >>>>>>>
> >>>>>>> Thank you point this, already add it again.
> >>>>>>>
> >>>>>>> The KIP already updated, please take a look.
> >>>>>>>
> >>>>>>> ---
> >>>>>>> Vito
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>
> >>
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Good question about `StreamsRebalancingException` -- when this KIP was
started, KIP-535 was not on the horizon yet.

What I am wondering is, if we should allow people to opt-in into
querying during a rebalance, or to be more precise during a restore (if
a state store is not migrated, it will be up-to-date during a rebalance
and can be queried returning correct, ie, non-stall, data)?

Otherwise, if people want to get only correct results, ie, they never
want to query stall state, they have no way to implement it, because
they are always subject to a race condition.

For this case, we could have a `StateStoreIsRecoveringException` (or
similar) that is only throw during a restore phases (and people can
opt-in / opt-out if this exception should be throws or not, ie, if they
want to query stall state during recovery or not).

It's unclear to me though atm, how a user would opt-in/opt-out and what
the default should be (maybe better to throw the exception by default to
have strong consistency guarantees by default?)


-Matthias


On 1/9/20 11:35 AM, Vinoth Chandar wrote:
> +1 on merging `StreamsNotRunningException` and `StateStoreNotAvailableException`, both exceptions are fatal anyway. IMO its best to have these exceptions be about the state store (and not streams state), to easier understanding. 
> 
> Additionally, KIP-535 allows for querying of state stores in rebalancing state. So do we need the StreamsRebalancingException? 
> 
> 
> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io> wrote: 
>> Sorry that I dropped the ball on this...
>>
>> Thanks for updating the KIP. Overall LGTM now. Feel free to start a VOTE
>> thread.
>>
>> What is still unclear to me is, what we gain by having both
>> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
>> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
>> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
>> state store is closed on not -- I can't query it anyway? Maybe I miss
>> something thought?
>>
>>
>> -Matthias
>>
>>
>> On 11/3/19 6:07 PM, Vito Jeng wrote:
>>> Sorry for the late reply, thanks for the review.
>>>
>>>
>>>> About `StateStoreMigratedException`:
>>>>
>>>> Why is it only thrown if the state is REBALANCING? A store might be
>>>> migrated during a rebalance, and Kafka Streams might resume back to
>>>> RUNNING state and afterward somebody tries to use an old store handle.
>>>> Also, if state is REBALANCING, should we throw
>>>> `StreamThreadRebalancingException`? Hence, I think
>>>> `StateStoreMigratedException` does only make sense during `RUNNING` state.
>>>>
>>>
>>> Thank you point this, already updated.
>>>
>>>
>>> Why do we need to distinguish between `KafkaStreamsNotRunningException`
>>>> and `StateStoreNotAvailableException`?
>>>>
>>>
>>> `KafkaStreamsNotRunningException` may be caused by various reasons, I think
>>> it would be helpful that the
>>> user can distinguish whether it is caused by the state store closed.
>>> (Maybe I am wrong...)
>>>
>>>
>>> Last, why do we distinguish between `KafkaStreams` instance and
>>>> `StreamsThread`? To me, it seems we should always refer to the instance,
>>>> because that is the level of granularity in which we enable/disable IQ atm.
>>>>
>>>
>>> Totally agree. Do you mean the naming of state store exceptions?
>>> I don't have special reason to distinguish these two.
>>> Your suggestion look more reasonable for the exception naming.
>>>
>>>
>>> Last, for `StateStoreMigratedException`, I would add that a user need to
>>>> rediscover the store and cannot blindly retry as the store handle is
>>>> invalid and a new store handle must be retrieved. That is a difference
>>>> to `StreamThreadRebalancingException` that allows for "blind" retries
>>>> that either resolve (if the store is still on the same instance after
>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if the
>>>> store was migrated away during rebalancing).
>>>>
>>>
>>> Nice, it's great! Thank you.
>>>
>>>
>>> The KIP already updated, please take a look. :)
>>>
>>>
>>>
>>> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>> Any update on this KIP?
>>>>
>>>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>>>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
>>>>>
>>>>> About `StateStoreMigratedException`:
>>>>>
>>>>> Why is it only thrown if the state is REBALANCING? A store might be
>>>>> migrated during a rebalance, and Kafka Streams might resume back to
>>>>> RUNNING state and afterward somebody tries to use an old store handle.
>>>>> Also, if state is REBALANCING, should we throw
>>>>> `StreamThreadRebalancingException`? Hence, I think
>>>>> `StateStoreMigratedException` does only make sense during `RUNNING`
>>>> state.
>>>>>
>>>>>
>>>>> Why do we need to distinguish between `KafkaStreamsNotRunningException`
>>>>> and `StateStoreNotAvailableException`?
>>>>>
>>>>>
>>>>> Last, why do we distinguish between `KafkaStreams` instance and
>>>>> `StreamsThread`? To me, it seems we should always refer to the instance,
>>>>> because that is the level of granularity in which we enable/disable IQ
>>>> atm.
>>>>>
>>>>>
>>>>> Last, for `StateStoreMigratedException`, I would add that a user need to
>>>>> rediscover the store and cannot blindly retry as the store handle is
>>>>> invalid and a new store handle must be retrieved. That is a difference
>>>>> to `StreamThreadRebalancingException` that allows for "blind" retries
>>>>> that either resolve (if the store is still on the same instance after
>>>>> rebalancing finishes, or changes to `StateStoreMigratedException` if the
>>>>> store was migrated away during rebalancing).
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>>>>>> My bad. The short link `https://shorturl.at/CDNT9`
>>>> <https://shorturl.at/CDNT9>
>>>>>> <https://shorturl.at/CDNT9> seems incorrect.
>>>>>>
>>>>>> Please use the following instead: https://shorturl.at/bkKQU
>>>>>>
>>>>>>
>>>>>> ---
>>>>>> Vito
>>>>>>
>>>>>>
>>>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw> wrote:
>>>>>>
>>>>>>> Thanks, Matthias!
>>>>>>>
>>>>>>>> About `StreamThreadNotStartedException`:
>>>>>>>
>>>>>>> Thank you for explanation. I agree with your opinion.
>>>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>>>>>>> `StreamThreadNotStartedException`.
>>>>>>>
>>>>>>> For the case that corresponding thread crashes after we handed out the
>>>>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
>>>>>>> `StateStoreMigratedException`.
>>>>>>> In `StreamThreadStateStoreProvider`, we would throw
>>>>>>> `KafkaStreamsNotRunningException` when stream thread is not running(
>>>>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
>>>>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not need
>>>> to
>>>>>>> add a new type for this case. Does that make sense?
>>>>>>>
>>>>>>>
>>>>>>>> About `KafkaStreamsNotRunningException` vs
>>>>>>> `StreamThreadNotRunningException`:
>>>>>>>
>>>>>>> I understand your point. I rename `StreamThreadNotRunningException` to
>>>>>>> `KafkaStreamsNotRunningException`.
>>>>>>>
>>>>>>>
>>>>>>> About check unknown state store names:
>>>>>>> Thank you for the hint. I add a new type `UnknownStateStoreException`
>>>> for
>>>>>>> this case.
>>>>>>>
>>>>>>>
>>>>>>>> Also, we should still have fatal exception
>>>>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
>>>>>>>
>>>>>>> Thank you point this, already add it again.
>>>>>>>
>>>>>>> The KIP already updated, please take a look.
>>>>>>>
>>>>>>> ---
>>>>>>> Vito
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>
>>
>>


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Hi, Matthias & Vinoth,

Thanks for the feedback.

> What is still unclear to me is, what we gain by having both
> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> state store is closed on not -- I can't query it anyway? Maybe I miss
> something thought?

Yes, both `StreamsNotRunningException` and
`StateStoreNotAvailableException` are fatal exception.
But `StateStoreNotAvailableException` is fatal exception about state store
related.
I think it would be helpful that if user need to distinguish these two
different case to handle it.

I'm not very sure, does that make sense?


---
Vito


On Fri, Jan 10, 2020 at 3:35 AM Vinoth Chandar <vi...@apache.org> wrote:

> +1 on merging `StreamsNotRunningException` and
> `StateStoreNotAvailableException`, both exceptions are fatal anyway. IMO
> its best to have these exceptions be about the state store (and not streams
> state), to easier understanding.
>
> Additionally, KIP-535 allows for querying of state stores in rebalancing
> state. So do we need the StreamsRebalancingException?
>
>
> On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io> wrote:
> > Sorry that I dropped the ball on this...
> >
> > Thanks for updating the KIP. Overall LGTM now. Feel free to start a VOTE
> > thread.
> >
> > What is still unclear to me is, what we gain by having both
> > `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
> > exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> > NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> > state store is closed on not -- I can't query it anyway? Maybe I miss
> > something thought?
> >
> >
> > -Matthias
> >
> >
> > On 11/3/19 6:07 PM, Vito Jeng wrote:
> > > Sorry for the late reply, thanks for the review.
> > >
> > >
> > >> About `StateStoreMigratedException`:
> > >>
> > >> Why is it only thrown if the state is REBALANCING? A store might be
> > >> migrated during a rebalance, and Kafka Streams might resume back to
> > >> RUNNING state and afterward somebody tries to use an old store handle.
> > >> Also, if state is REBALANCING, should we throw
> > >> `StreamThreadRebalancingException`? Hence, I think
> > >> `StateStoreMigratedException` does only make sense during `RUNNING`
> state.
> > >>
> > >
> > > Thank you point this, already updated.
> > >
> > >
> > > Why do we need to distinguish between `KafkaStreamsNotRunningException`
> > >> and `StateStoreNotAvailableException`?
> > >>
> > >
> > > `KafkaStreamsNotRunningException` may be caused by various reasons, I
> think
> > > it would be helpful that the
> > > user can distinguish whether it is caused by the state store closed.
> > > (Maybe I am wrong...)
> > >
> > >
> > > Last, why do we distinguish between `KafkaStreams` instance and
> > >> `StreamsThread`? To me, it seems we should always refer to the
> instance,
> > >> because that is the level of granularity in which we enable/disable
> IQ atm.
> > >>
> > >
> > > Totally agree. Do you mean the naming of state store exceptions?
> > > I don't have special reason to distinguish these two.
> > > Your suggestion look more reasonable for the exception naming.
> > >
> > >
> > > Last, for `StateStoreMigratedException`, I would add that a user need
> to
> > >> rediscover the store and cannot blindly retry as the store handle is
> > >> invalid and a new store handle must be retrieved. That is a difference
> > >> to `StreamThreadRebalancingException` that allows for "blind" retries
> > >> that either resolve (if the store is still on the same instance after
> > >> rebalancing finishes, or changes to `StateStoreMigratedException` if
> the
> > >> store was migrated away during rebalancing).
> > >>
> > >
> > > Nice, it's great! Thank you.
> > >
> > >
> > > The KIP already updated, please take a look. :)
> > >
> > >
> > >
> > > On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <matthias@confluent.io
> >
> > > wrote:
> > >
> > >> Any update on this KIP?
> > >>
> > >> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> > >>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> > >>>
> > >>> About `StateStoreMigratedException`:
> > >>>
> > >>> Why is it only thrown if the state is REBALANCING? A store might be
> > >>> migrated during a rebalance, and Kafka Streams might resume back to
> > >>> RUNNING state and afterward somebody tries to use an old store
> handle.
> > >>> Also, if state is REBALANCING, should we throw
> > >>> `StreamThreadRebalancingException`? Hence, I think
> > >>> `StateStoreMigratedException` does only make sense during `RUNNING`
> > >> state.
> > >>>
> > >>>
> > >>> Why do we need to distinguish between
> `KafkaStreamsNotRunningException`
> > >>> and `StateStoreNotAvailableException`?
> > >>>
> > >>>
> > >>> Last, why do we distinguish between `KafkaStreams` instance and
> > >>> `StreamsThread`? To me, it seems we should always refer to the
> instance,
> > >>> because that is the level of granularity in which we enable/disable
> IQ
> > >> atm.
> > >>>
> > >>>
> > >>> Last, for `StateStoreMigratedException`, I would add that a user
> need to
> > >>> rediscover the store and cannot blindly retry as the store handle is
> > >>> invalid and a new store handle must be retrieved. That is a
> difference
> > >>> to `StreamThreadRebalancingException` that allows for "blind" retries
> > >>> that either resolve (if the store is still on the same instance after
> > >>> rebalancing finishes, or changes to `StateStoreMigratedException` if
> the
> > >>> store was migrated away during rebalancing).
> > >>>
> > >>>
> > >>>
> > >>> -Matthias
> > >>>
> > >>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> > >>>> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> > >> <https://shorturl.at/CDNT9>
> > >>>> <https://shorturl.at/CDNT9> seems incorrect.
> > >>>>
> > >>>> Please use the following instead: https://shorturl.at/bkKQU
> > >>>>
> > >>>>
> > >>>> ---
> > >>>> Vito
> > >>>>
> > >>>>
> > >>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw>
> wrote:
> > >>>>
> > >>>>> Thanks, Matthias!
> > >>>>>
> > >>>>>> About `StreamThreadNotStartedException`:
> > >>>>>
> > >>>>> Thank you for explanation. I agree with your opinion.
> > >>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> > >>>>> `StreamThreadNotStartedException`.
> > >>>>>
> > >>>>> For the case that corresponding thread crashes after we handed out
> the
> > >>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
> > >>>>> `StateStoreMigratedException`.
> > >>>>> In `StreamThreadStateStoreProvider`, we would throw
> > >>>>> `KafkaStreamsNotRunningException` when stream thread is not
> running(
> > >>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException`
> when
> > >>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not
> need
> > >> to
> > >>>>> add a new type for this case. Does that make sense?
> > >>>>>
> > >>>>>
> > >>>>>> About `KafkaStreamsNotRunningException` vs
> > >>>>> `StreamThreadNotRunningException`:
> > >>>>>
> > >>>>> I understand your point. I rename
> `StreamThreadNotRunningException` to
> > >>>>> `KafkaStreamsNotRunningException`.
> > >>>>>
> > >>>>>
> > >>>>> About check unknown state store names:
> > >>>>> Thank you for the hint. I add a new type
> `UnknownStateStoreException`
> > >> for
> > >>>>> this case.
> > >>>>>
> > >>>>>
> > >>>>>> Also, we should still have fatal exception
> > >>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
> > >>>>>
> > >>>>> Thank you point this, already add it again.
> > >>>>>
> > >>>>> The KIP already updated, please take a look.
> > >>>>>
> > >>>>> ---
> > >>>>> Vito
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >>
> > >
> >
> >
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vinoth Chandar <vi...@apache.org>.
+1 on merging `StreamsNotRunningException` and `StateStoreNotAvailableException`, both exceptions are fatal anyway. IMO its best to have these exceptions be about the state store (and not streams state), to easier understanding. 

Additionally, KIP-535 allows for querying of state stores in rebalancing state. So do we need the StreamsRebalancingException? 


On 2020/01/09 03:38:11, "Matthias J. Sax" <ma...@confluent.io> wrote: 
> Sorry that I dropped the ball on this...
> 
> Thanks for updating the KIP. Overall LGTM now. Feel free to start a VOTE
> thread.
> 
> What is still unclear to me is, what we gain by having both
> `StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
> exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
> NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
> state store is closed on not -- I can't query it anyway? Maybe I miss
> something thought?
> 
> 
> -Matthias
> 
> 
> On 11/3/19 6:07 PM, Vito Jeng wrote:
> > Sorry for the late reply, thanks for the review.
> > 
> > 
> >> About `StateStoreMigratedException`:
> >>
> >> Why is it only thrown if the state is REBALANCING? A store might be
> >> migrated during a rebalance, and Kafka Streams might resume back to
> >> RUNNING state and afterward somebody tries to use an old store handle.
> >> Also, if state is REBALANCING, should we throw
> >> `StreamThreadRebalancingException`? Hence, I think
> >> `StateStoreMigratedException` does only make sense during `RUNNING` state.
> >>
> > 
> > Thank you point this, already updated.
> > 
> > 
> > Why do we need to distinguish between `KafkaStreamsNotRunningException`
> >> and `StateStoreNotAvailableException`?
> >>
> > 
> > `KafkaStreamsNotRunningException` may be caused by various reasons, I think
> > it would be helpful that the
> > user can distinguish whether it is caused by the state store closed.
> > (Maybe I am wrong...)
> > 
> > 
> > Last, why do we distinguish between `KafkaStreams` instance and
> >> `StreamsThread`? To me, it seems we should always refer to the instance,
> >> because that is the level of granularity in which we enable/disable IQ atm.
> >>
> > 
> > Totally agree. Do you mean the naming of state store exceptions?
> > I don't have special reason to distinguish these two.
> > Your suggestion look more reasonable for the exception naming.
> > 
> > 
> > Last, for `StateStoreMigratedException`, I would add that a user need to
> >> rediscover the store and cannot blindly retry as the store handle is
> >> invalid and a new store handle must be retrieved. That is a difference
> >> to `StreamThreadRebalancingException` that allows for "blind" retries
> >> that either resolve (if the store is still on the same instance after
> >> rebalancing finishes, or changes to `StateStoreMigratedException` if the
> >> store was migrated away during rebalancing).
> >>
> > 
> > Nice, it's great! Thank you.
> > 
> > 
> > The KIP already updated, please take a look. :)
> > 
> > 
> > 
> > On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> > 
> >> Any update on this KIP?
> >>
> >> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> >>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> >>>
> >>> About `StateStoreMigratedException`:
> >>>
> >>> Why is it only thrown if the state is REBALANCING? A store might be
> >>> migrated during a rebalance, and Kafka Streams might resume back to
> >>> RUNNING state and afterward somebody tries to use an old store handle.
> >>> Also, if state is REBALANCING, should we throw
> >>> `StreamThreadRebalancingException`? Hence, I think
> >>> `StateStoreMigratedException` does only make sense during `RUNNING`
> >> state.
> >>>
> >>>
> >>> Why do we need to distinguish between `KafkaStreamsNotRunningException`
> >>> and `StateStoreNotAvailableException`?
> >>>
> >>>
> >>> Last, why do we distinguish between `KafkaStreams` instance and
> >>> `StreamsThread`? To me, it seems we should always refer to the instance,
> >>> because that is the level of granularity in which we enable/disable IQ
> >> atm.
> >>>
> >>>
> >>> Last, for `StateStoreMigratedException`, I would add that a user need to
> >>> rediscover the store and cannot blindly retry as the store handle is
> >>> invalid and a new store handle must be retrieved. That is a difference
> >>> to `StreamThreadRebalancingException` that allows for "blind" retries
> >>> that either resolve (if the store is still on the same instance after
> >>> rebalancing finishes, or changes to `StateStoreMigratedException` if the
> >>> store was migrated away during rebalancing).
> >>>
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 8/9/19 10:20 AM, Vito Jeng wrote:
> >>>> My bad. The short link `https://shorturl.at/CDNT9`
> >> <https://shorturl.at/CDNT9>
> >>>> <https://shorturl.at/CDNT9> seems incorrect.
> >>>>
> >>>> Please use the following instead: https://shorturl.at/bkKQU
> >>>>
> >>>>
> >>>> ---
> >>>> Vito
> >>>>
> >>>>
> >>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> >>>>
> >>>>> Thanks, Matthias!
> >>>>>
> >>>>>> About `StreamThreadNotStartedException`:
> >>>>>
> >>>>> Thank you for explanation. I agree with your opinion.
> >>>>> `CompositeReadOnlyXxxStore#get()` would never throw
> >>>>> `StreamThreadNotStartedException`.
> >>>>>
> >>>>> For the case that corresponding thread crashes after we handed out the
> >>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
> >>>>> `StateStoreMigratedException`.
> >>>>> In `StreamThreadStateStoreProvider`, we would throw
> >>>>> `KafkaStreamsNotRunningException` when stream thread is not running(
> >>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
> >>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not need
> >> to
> >>>>> add a new type for this case. Does that make sense?
> >>>>>
> >>>>>
> >>>>>> About `KafkaStreamsNotRunningException` vs
> >>>>> `StreamThreadNotRunningException`:
> >>>>>
> >>>>> I understand your point. I rename `StreamThreadNotRunningException` to
> >>>>> `KafkaStreamsNotRunningException`.
> >>>>>
> >>>>>
> >>>>> About check unknown state store names:
> >>>>> Thank you for the hint. I add a new type `UnknownStateStoreException`
> >> for
> >>>>> this case.
> >>>>>
> >>>>>
> >>>>>> Also, we should still have fatal exception
> >>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
> >>>>>
> >>>>> Thank you point this, already add it again.
> >>>>>
> >>>>> The KIP already updated, please take a look.
> >>>>>
> >>>>> ---
> >>>>> Vito
> >>>>>
> >>>>
> >>>
> >>
> >>
> > 
> 
> 

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Sorry that I dropped the ball on this...

Thanks for updating the KIP. Overall LGTM now. Feel free to start a VOTE
thread.

What is still unclear to me is, what we gain by having both
`StreamsNotRunningException` and `StateStoreNotAvailableException`. Both
exception are thrown when KafkaStreams is in state PENDING_SHUTDOWN /
NOT_RUNNING / ERROR. Hence, as a user what do I gain to know if the
state store is closed on not -- I can't query it anyway? Maybe I miss
something thought?


-Matthias


On 11/3/19 6:07 PM, Vito Jeng wrote:
> Sorry for the late reply, thanks for the review.
> 
> 
>> About `StateStoreMigratedException`:
>>
>> Why is it only thrown if the state is REBALANCING? A store might be
>> migrated during a rebalance, and Kafka Streams might resume back to
>> RUNNING state and afterward somebody tries to use an old store handle.
>> Also, if state is REBALANCING, should we throw
>> `StreamThreadRebalancingException`? Hence, I think
>> `StateStoreMigratedException` does only make sense during `RUNNING` state.
>>
> 
> Thank you point this, already updated.
> 
> 
> Why do we need to distinguish between `KafkaStreamsNotRunningException`
>> and `StateStoreNotAvailableException`?
>>
> 
> `KafkaStreamsNotRunningException` may be caused by various reasons, I think
> it would be helpful that the
> user can distinguish whether it is caused by the state store closed.
> (Maybe I am wrong...)
> 
> 
> Last, why do we distinguish between `KafkaStreams` instance and
>> `StreamsThread`? To me, it seems we should always refer to the instance,
>> because that is the level of granularity in which we enable/disable IQ atm.
>>
> 
> Totally agree. Do you mean the naming of state store exceptions?
> I don't have special reason to distinguish these two.
> Your suggestion look more reasonable for the exception naming.
> 
> 
> Last, for `StateStoreMigratedException`, I would add that a user need to
>> rediscover the store and cannot blindly retry as the store handle is
>> invalid and a new store handle must be retrieved. That is a difference
>> to `StreamThreadRebalancingException` that allows for "blind" retries
>> that either resolve (if the store is still on the same instance after
>> rebalancing finishes, or changes to `StateStoreMigratedException` if the
>> store was migrated away during rebalancing).
>>
> 
> Nice, it's great! Thank you.
> 
> 
> The KIP already updated, please take a look. :)
> 
> 
> 
> On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> Any update on this KIP?
>>
>> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
>>> Sorry for the late reply. The 2.4 deadline kept us quite busy.
>>>
>>> About `StateStoreMigratedException`:
>>>
>>> Why is it only thrown if the state is REBALANCING? A store might be
>>> migrated during a rebalance, and Kafka Streams might resume back to
>>> RUNNING state and afterward somebody tries to use an old store handle.
>>> Also, if state is REBALANCING, should we throw
>>> `StreamThreadRebalancingException`? Hence, I think
>>> `StateStoreMigratedException` does only make sense during `RUNNING`
>> state.
>>>
>>>
>>> Why do we need to distinguish between `KafkaStreamsNotRunningException`
>>> and `StateStoreNotAvailableException`?
>>>
>>>
>>> Last, why do we distinguish between `KafkaStreams` instance and
>>> `StreamsThread`? To me, it seems we should always refer to the instance,
>>> because that is the level of granularity in which we enable/disable IQ
>> atm.
>>>
>>>
>>> Last, for `StateStoreMigratedException`, I would add that a user need to
>>> rediscover the store and cannot blindly retry as the store handle is
>>> invalid and a new store handle must be retrieved. That is a difference
>>> to `StreamThreadRebalancingException` that allows for "blind" retries
>>> that either resolve (if the store is still on the same instance after
>>> rebalancing finishes, or changes to `StateStoreMigratedException` if the
>>> store was migrated away during rebalancing).
>>>
>>>
>>>
>>> -Matthias
>>>
>>> On 8/9/19 10:20 AM, Vito Jeng wrote:
>>>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9>
>>>> <https://shorturl.at/CDNT9> seems incorrect.
>>>>
>>>> Please use the following instead: https://shorturl.at/bkKQU
>>>>
>>>>
>>>> ---
>>>> Vito
>>>>
>>>>
>>>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw> wrote:
>>>>
>>>>> Thanks, Matthias!
>>>>>
>>>>>> About `StreamThreadNotStartedException`:
>>>>>
>>>>> Thank you for explanation. I agree with your opinion.
>>>>> `CompositeReadOnlyXxxStore#get()` would never throw
>>>>> `StreamThreadNotStartedException`.
>>>>>
>>>>> For the case that corresponding thread crashes after we handed out the
>>>>> store handle. We may throw `KafkaStreamsNotRunningException` or
>>>>> `StateStoreMigratedException`.
>>>>> In `StreamThreadStateStoreProvider`, we would throw
>>>>> `KafkaStreamsNotRunningException` when stream thread is not running(
>>>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
>>>>> store is closed(https://shorturl.at/hrvAN). So I think we do not need
>> to
>>>>> add a new type for this case. Does that make sense?
>>>>>
>>>>>
>>>>>> About `KafkaStreamsNotRunningException` vs
>>>>> `StreamThreadNotRunningException`:
>>>>>
>>>>> I understand your point. I rename `StreamThreadNotRunningException` to
>>>>> `KafkaStreamsNotRunningException`.
>>>>>
>>>>>
>>>>> About check unknown state store names:
>>>>> Thank you for the hint. I add a new type `UnknownStateStoreException`
>> for
>>>>> this case.
>>>>>
>>>>>
>>>>>> Also, we should still have fatal exception
>>>>> `StateStoreNotAvailableException`? Not sure why you remove it?
>>>>>
>>>>> Thank you point this, already add it again.
>>>>>
>>>>> The KIP already updated, please take a look.
>>>>>
>>>>> ---
>>>>> Vito
>>>>>
>>>>
>>>
>>
>>
> 


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
Sorry for the late reply, thanks for the review.


> About `StateStoreMigratedException`:
>
> Why is it only thrown if the state is REBALANCING? A store might be
> migrated during a rebalance, and Kafka Streams might resume back to
> RUNNING state and afterward somebody tries to use an old store handle.
> Also, if state is REBALANCING, should we throw
> `StreamThreadRebalancingException`? Hence, I think
> `StateStoreMigratedException` does only make sense during `RUNNING` state.
>

Thank you point this, already updated.


Why do we need to distinguish between `KafkaStreamsNotRunningException`
> and `StateStoreNotAvailableException`?
>

`KafkaStreamsNotRunningException` may be caused by various reasons, I think
it would be helpful that the
user can distinguish whether it is caused by the state store closed.
(Maybe I am wrong...)


Last, why do we distinguish between `KafkaStreams` instance and
> `StreamsThread`? To me, it seems we should always refer to the instance,
> because that is the level of granularity in which we enable/disable IQ atm.
>

Totally agree. Do you mean the naming of state store exceptions?
I don't have special reason to distinguish these two.
Your suggestion look more reasonable for the exception naming.


Last, for `StateStoreMigratedException`, I would add that a user need to
> rediscover the store and cannot blindly retry as the store handle is
> invalid and a new store handle must be retrieved. That is a difference
> to `StreamThreadRebalancingException` that allows for "blind" retries
> that either resolve (if the store is still on the same instance after
> rebalancing finishes, or changes to `StateStoreMigratedException` if the
> store was migrated away during rebalancing).
>

Nice, it's great! Thank you.


The KIP already updated, please take a look. :)



On Wed, Oct 23, 2019 at 1:48 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Any update on this KIP?
>
> On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> > Sorry for the late reply. The 2.4 deadline kept us quite busy.
> >
> > About `StateStoreMigratedException`:
> >
> > Why is it only thrown if the state is REBALANCING? A store might be
> > migrated during a rebalance, and Kafka Streams might resume back to
> > RUNNING state and afterward somebody tries to use an old store handle.
> > Also, if state is REBALANCING, should we throw
> > `StreamThreadRebalancingException`? Hence, I think
> > `StateStoreMigratedException` does only make sense during `RUNNING`
> state.
> >
> >
> > Why do we need to distinguish between `KafkaStreamsNotRunningException`
> > and `StateStoreNotAvailableException`?
> >
> >
> > Last, why do we distinguish between `KafkaStreams` instance and
> > `StreamsThread`? To me, it seems we should always refer to the instance,
> > because that is the level of granularity in which we enable/disable IQ
> atm.
> >
> >
> > Last, for `StateStoreMigratedException`, I would add that a user need to
> > rediscover the store and cannot blindly retry as the store handle is
> > invalid and a new store handle must be retrieved. That is a difference
> > to `StreamThreadRebalancingException` that allows for "blind" retries
> > that either resolve (if the store is still on the same instance after
> > rebalancing finishes, or changes to `StateStoreMigratedException` if the
> > store was migrated away during rebalancing).
> >
> >
> >
> > -Matthias
> >
> > On 8/9/19 10:20 AM, Vito Jeng wrote:
> >> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9>
> >> <https://shorturl.at/CDNT9> seems incorrect.
> >>
> >> Please use the following instead: https://shorturl.at/bkKQU
> >>
> >>
> >> ---
> >> Vito
> >>
> >>
> >> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> >>
> >>> Thanks, Matthias!
> >>>
> >>>> About `StreamThreadNotStartedException`:
> >>>
> >>> Thank you for explanation. I agree with your opinion.
> >>> `CompositeReadOnlyXxxStore#get()` would never throw
> >>> `StreamThreadNotStartedException`.
> >>>
> >>> For the case that corresponding thread crashes after we handed out the
> >>> store handle. We may throw `KafkaStreamsNotRunningException` or
> >>> `StateStoreMigratedException`.
> >>> In `StreamThreadStateStoreProvider`, we would throw
> >>> `KafkaStreamsNotRunningException` when stream thread is not running(
> >>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
> >>> store is closed(https://shorturl.at/hrvAN). So I think we do not need
> to
> >>> add a new type for this case. Does that make sense?
> >>>
> >>>
> >>>> About `KafkaStreamsNotRunningException` vs
> >>> `StreamThreadNotRunningException`:
> >>>
> >>> I understand your point. I rename `StreamThreadNotRunningException` to
> >>> `KafkaStreamsNotRunningException`.
> >>>
> >>>
> >>> About check unknown state store names:
> >>> Thank you for the hint. I add a new type `UnknownStateStoreException`
> for
> >>> this case.
> >>>
> >>>
> >>>> Also, we should still have fatal exception
> >>> `StateStoreNotAvailableException`? Not sure why you remove it?
> >>>
> >>> Thank you point this, already add it again.
> >>>
> >>> The KIP already updated, please take a look.
> >>>
> >>> ---
> >>> Vito
> >>>
> >>
> >
>
>

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Any update on this KIP?

On 10/7/19 3:35 PM, Matthias J. Sax wrote:
> Sorry for the late reply. The 2.4 deadline kept us quite busy.
> 
> About `StateStoreMigratedException`:
> 
> Why is it only thrown if the state is REBALANCING? A store might be
> migrated during a rebalance, and Kafka Streams might resume back to
> RUNNING state and afterward somebody tries to use an old store handle.
> Also, if state is REBALANCING, should we throw
> `StreamThreadRebalancingException`? Hence, I think
> `StateStoreMigratedException` does only make sense during `RUNNING` state.
> 
> 
> Why do we need to distinguish between `KafkaStreamsNotRunningException`
> and `StateStoreNotAvailableException`?
> 
> 
> Last, why do we distinguish between `KafkaStreams` instance and
> `StreamsThread`? To me, it seems we should always refer to the instance,
> because that is the level of granularity in which we enable/disable IQ atm.
> 
> 
> Last, for `StateStoreMigratedException`, I would add that a user need to
> rediscover the store and cannot blindly retry as the store handle is
> invalid and a new store handle must be retrieved. That is a difference
> to `StreamThreadRebalancingException` that allows for "blind" retries
> that either resolve (if the store is still on the same instance after
> rebalancing finishes, or changes to `StateStoreMigratedException` if the
> store was migrated away during rebalancing).
> 
> 
> 
> -Matthias
> 
> On 8/9/19 10:20 AM, Vito Jeng wrote:
>> My bad. The short link `https://shorturl.at/CDNT9`
>> <https://shorturl.at/CDNT9> seems incorrect.
>>
>> Please use the following instead: https://shorturl.at/bkKQU
>>
>>
>> ---
>> Vito
>>
>>
>> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw> wrote:
>>
>>> Thanks, Matthias!
>>>
>>>> About `StreamThreadNotStartedException`:
>>>
>>> Thank you for explanation. I agree with your opinion.
>>> `CompositeReadOnlyXxxStore#get()` would never throw
>>> `StreamThreadNotStartedException`.
>>>
>>> For the case that corresponding thread crashes after we handed out the
>>> store handle. We may throw `KafkaStreamsNotRunningException` or
>>> `StateStoreMigratedException`.
>>> In `StreamThreadStateStoreProvider`, we would throw
>>> `KafkaStreamsNotRunningException` when stream thread is not running(
>>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
>>> store is closed(https://shorturl.at/hrvAN). So I think we do not need to
>>> add a new type for this case. Does that make sense?
>>>
>>>
>>>> About `KafkaStreamsNotRunningException` vs
>>> `StreamThreadNotRunningException`:
>>>
>>> I understand your point. I rename `StreamThreadNotRunningException` to
>>> `KafkaStreamsNotRunningException`.
>>>
>>>
>>> About check unknown state store names:
>>> Thank you for the hint. I add a new type `UnknownStateStoreException` for
>>> this case.
>>>
>>>
>>>> Also, we should still have fatal exception
>>> `StateStoreNotAvailableException`? Not sure why you remove it?
>>>
>>> Thank you point this, already add it again.
>>>
>>> The KIP already updated, please take a look.
>>>
>>> ---
>>> Vito
>>>
>>
> 


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Sorry for the late reply. The 2.4 deadline kept us quite busy.

About `StateStoreMigratedException`:

Why is it only thrown if the state is REBALANCING? A store might be
migrated during a rebalance, and Kafka Streams might resume back to
RUNNING state and afterward somebody tries to use an old store handle.
Also, if state is REBALANCING, should we throw
`StreamThreadRebalancingException`? Hence, I think
`StateStoreMigratedException` does only make sense during `RUNNING` state.


Why do we need to distinguish between `KafkaStreamsNotRunningException`
and `StateStoreNotAvailableException`?


Last, why do we distinguish between `KafkaStreams` instance and
`StreamsThread`? To me, it seems we should always refer to the instance,
because that is the level of granularity in which we enable/disable IQ atm.


Last, for `StateStoreMigratedException`, I would add that a user need to
rediscover the store and cannot blindly retry as the store handle is
invalid and a new store handle must be retrieved. That is a difference
to `StreamThreadRebalancingException` that allows for "blind" retries
that either resolve (if the store is still on the same instance after
rebalancing finishes, or changes to `StateStoreMigratedException` if the
store was migrated away during rebalancing).



-Matthias

On 8/9/19 10:20 AM, Vito Jeng wrote:
> My bad. The short link `https://shorturl.at/CDNT9`
> <https://shorturl.at/CDNT9> seems incorrect.
> 
> Please use the following instead: https://shorturl.at/bkKQU
> 
> 
> ---
> Vito
> 
> 
> On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw> wrote:
> 
>> Thanks, Matthias!
>>
>>> About `StreamThreadNotStartedException`:
>>
>> Thank you for explanation. I agree with your opinion.
>> `CompositeReadOnlyXxxStore#get()` would never throw
>> `StreamThreadNotStartedException`.
>>
>> For the case that corresponding thread crashes after we handed out the
>> store handle. We may throw `KafkaStreamsNotRunningException` or
>> `StateStoreMigratedException`.
>> In `StreamThreadStateStoreProvider`, we would throw
>> `KafkaStreamsNotRunningException` when stream thread is not running(
>> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
>> store is closed(https://shorturl.at/hrvAN). So I think we do not need to
>> add a new type for this case. Does that make sense?
>>
>>
>>> About `KafkaStreamsNotRunningException` vs
>> `StreamThreadNotRunningException`:
>>
>> I understand your point. I rename `StreamThreadNotRunningException` to
>> `KafkaStreamsNotRunningException`.
>>
>>
>> About check unknown state store names:
>> Thank you for the hint. I add a new type `UnknownStateStoreException` for
>> this case.
>>
>>
>>> Also, we should still have fatal exception
>> `StateStoreNotAvailableException`? Not sure why you remove it?
>>
>> Thank you point this, already add it again.
>>
>> The KIP already updated, please take a look.
>>
>> ---
>> Vito
>>
> 


Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

Posted by Vito Jeng <vi...@is-land.com.tw>.
My bad. The short link `https://shorturl.at/CDNT9`
<https://shorturl.at/CDNT9> seems incorrect.

Please use the following instead: https://shorturl.at/bkKQU


---
Vito


On Fri, Aug 9, 2019 at 10:53 AM Vito Jeng <vi...@is-land.com.tw> wrote:

> Thanks, Matthias!
>
> > About `StreamThreadNotStartedException`:
>
> Thank you for explanation. I agree with your opinion.
> `CompositeReadOnlyXxxStore#get()` would never throw
> `StreamThreadNotStartedException`.
>
> For the case that corresponding thread crashes after we handed out the
> store handle. We may throw `KafkaStreamsNotRunningException` or
> `StateStoreMigratedException`.
> In `StreamThreadStateStoreProvider`, we would throw
> `KafkaStreamsNotRunningException` when stream thread is not running(
> https://shorturl.at/CDNT9) or throw `StateStoreMigratedException` when
> store is closed(https://shorturl.at/hrvAN). So I think we do not need to
> add a new type for this case. Does that make sense?
>
>
> > About `KafkaStreamsNotRunningException` vs
> `StreamThreadNotRunningException`:
>
> I understand your point. I rename `StreamThreadNotRunningException` to
> `KafkaStreamsNotRunningException`.
>
>
> About check unknown state store names:
> Thank you for the hint. I add a new type `UnknownStateStoreException` for
> this case.
>
>
> > Also, we should still have fatal exception
> `StateStoreNotAvailableException`? Not sure why you remove it?
>
> Thank you point this, already add it again.
>
> The KIP already updated, please take a look.
>
> ---
> Vito
>