[DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-13 Thread Navinder Brar
Hi, Starting a discussion on the KIP to Allow state stores to serve stale reads during rebalance(https://cwiki.apache.org/confluence/display/KAFKA/KIP-535%3A+Allow+state+stores+to+serve+stale+reads+during+rebalance). Thanks & Regards,Navinder  LinkedIn

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-15 Thread Navinder Brar
standby replicas, which could be mostly caught up and the> >staleness window could be much smaller/tolerable. (once again the focus on> >KAFKA-8994)> >- Finally, we may need to introduce a configuration to control this. Some> >users may prefer errors to stale data. Can we

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-16 Thread Navinder Brar
m would be inconsistent for such a long time in practice. > > So, I still feel just limiting this to standby reads provides best > semantics. > > Just my 2c. Would love to see what others think as well. > > On Tue, Oct 15, 2019 at 5:34 AM Navinder Brar > wrote: > >> H

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-19 Thread Navinder Brar
Do we really need a per key configuration? or a new StreamsConfig is good enough? On Wed, Oct 16, 2019 at 8:31 PM Navinder Brar wrote: > @Vinoth, I have incorporated a few of the discussions we have had in the > KIP. > > In the current code, t0 and t1 serve queries from Active(Running

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-23 Thread Navinder Brar
>> is _the_ key point for this KIP. FWIW, I think that adding a new >>>>> method to StreamsMetadataState and deprecating the existing method is >>>>> the best way to go; we just can't change the return types of any >>>>> existing methods. >>&

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-24 Thread Navinder Brar
ive host > > metadata > > > (as it was before). We should also document this in that APIs javadoc, > > > given we have another method(s) that returns more host metadata now. > > > > > > 5.  While I see the point, the app/caller has to make two different >

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-25 Thread Navinder Brar
>>>>> 1. IIUC John intends to add (or we can do this in this KIP) lag >>>> information >>>>> to AssignmentInfo, which gets sent to every participant. >>>>> >>>>> 2. At-least I was under the assumption that it can be called per &g

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-25 Thread Navinder Brar
by Matthias >>>>> >>>>> 1. IIUC John intends to add (or we can do this in this KIP) lag >>>> information >>>>> to AssignmentInfo, which gets sent to every participant. >>>>> >>>>> 2. At-least I was under the ass

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-26 Thread Navinder Brar
t; to perform some filtering to separate the different tasks, so I don't feel > making two calls is a burden, and IMHO makes things more transparent for > the user. > If the final vote is for using an "isActive" field, I'm good with that as > well. > > Just my

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-30 Thread Navinder Brar
. On Saturday, 26 October, 2019, 05:41:44 pm IST, Navinder Brar wrote: >> Since there are two soft votes for separate active/standby API methods, I  >>also change my position on that. Fine with 2 separate methods. Once we >>remove the lag information from these APIs, return

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-10-31 Thread Navinder Brar
I'm working on it > > now. > > > > Thanks, > > John > > > > On Thu, Oct 31, 2019 at 11:42 AM Vinoth Chandar > wrote: > > > > > > Wondering if anyone has thoughts on these changes? I liked that the new > > > metadata f

Why standby tasks read from the StandbyTasks::checkpointedOffsets in assignStandbyPartitions()

2019-11-04 Thread Navinder Brar
Hi, Please let me know if this is not the correct forum to ask this. But I have a doubt, I was hoping someone can clear it for me. In TaskManager:: updateNewAndRestoringTasks(), the function  assignStandbyPartitions() gets called for all the running standby tasks where it populates the Map: check

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-05 Thread Navinder Brar
> > > > > > On Thu, Oct 31, 2019 at 12:58 PM John Roesler < > >> j...@confluent.io> > >> > > > wrote: > >> > > > > > > >> > > > > > > Hey Navinder, > >> > > > > > > > >>

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-06 Thread Navinder Brar
lee-Goldman < > >> sop...@confluent.io> > >>>> wrote: > >>>> > >>>>> Adding on to John's response to 3), can you clarify when and why > >>> exactly we > >>>>> cannot > >>>>> convert be

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-08 Thread Navinder Brar
hangelogPartitions" to differentiate it? Overall I think this would be a really good KIP to add to Streams, thank you so much! Guozhang On Wed, Nov 6, 2019 at 8:47 PM Navinder Brar wrote: > +1 on implementing offset based lag for now and push time-based lag to a > later point in time

Re: Why standby tasks read from the StandbyTasks::checkpointedOffsets in assignStandbyPartitions()

2019-11-10 Thread Navinder Brar
not re initialize the position on the consumer and hence it is still the task-manager's responsibility to set the right starting offset from the latest checkpoint file. If we did not do that, that should still be a bug. Guozhang On Sat, Nov 9, 2019 at 11:33 AM Navinder Brar wrote: Hi Guo

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-12 Thread Navinder Brar
gt; (1.1. + 1.2.) From my understanding `allMetadata()` (and other existing > methods) will only return the metadata of _active_ tasks for backward > compatibility reasons. If we would return standby metadata, existing > code would potentially "break" because the code might pi

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-14 Thread Navinder Brar
t; introducing a new class. > > >> This sort of thing would lead to non-standardized/potentially buggy > > client > > >> implementations, for something I expect the system would hand me > > directly. > > >> I don't personally feel introduc

[VOTE] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-14 Thread Navinder Brar
Hello all, I'd like to propose a vote for serving interactive queries during Rebalancing, as it is a big deal for applications looking for high availability. With this change, users will have control over the tradeoff between consistency and availability during serving. The full KIP is provided

Potential Bug in 2.3 version (leading to deletion of state directories)

2019-11-14 Thread Navinder Brar
Hi, We are facing a peculiar situation in the 2.3 version of Kafka Streams. First of all, I want to clarify if it is possible that a Stream Thread (say Stream Thread-1) which had got an assignment for a standby task (say 0_0) can change to Stream Thread-2 on the same host post rebalancing. The i

Re: [VOTE] KIP-626: Rename StreamsConfig config variable name

2020-06-18 Thread Navinder Brar
Hi Matthias, +1 non-binding. Thanks for the KIP. Thanks, Navinder  On Tuesday, 16 June, 2020, 06:34:19 pm IST, Bruno Cadonna wrote: Hi Matthias, Thank you for the KIP. I am +1 (non-binding). Said that, you have a typo in the "Public Interface" section. TOPLOGY_OPTIMIZATION_CONFIG

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-22 Thread Navinder Brar
Many Congratulations Boyang. Very well deserved. Regards,Navinder  On Tuesday, 23 June, 2020, 07:21:23 am IST, Matt Wang wrote: Congratulations, Boyang! -- Best, Matt Wang On 06/23/2020 07:59,Boyang Chen wrote: Thanks a lot everyone, I really appreciate the recognition, and hope

[DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-06-26 Thread Navinder Brar
Hi, KIP:  https://cwiki.apache.org/confluence/display/KAFKA/KIP-406%3A+GlobalStreamThread+should+honor+custom+reset+policy I have taken over this KIP since it has been dormant for a long time and this looks important for use-cases that have large global data, so rebuilding global stores from sc

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-06-28 Thread Navinder Brar
still propose to augment the Java API? Thanks, -John On Fri, Jun 26, 2020, at 23:52, Navinder Brar wrote: > Hi, > > KIP:  > https://cwiki.apache.org/confluence/display/KAFKA/KIP-406%3A+GlobalStreamThread+should+honor+custom+reset+policy > > I have taken over this KIP since it h

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-06-28 Thread Navinder Brar
, 29 June, 2020, 12:24:21 am IST, Navinder Brar wrote: Hi John, Thanks for your feedback.  1. I think there is some confusion on my first point, the enum I am sure we can use the same one but the external config which controls the resetting in global stream thread either we can the same

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-07-26 Thread Navinder Brar
not > currently using the config, but I thought we would pass the config through to > the client.  Can you confirm whether or not the existing config works for > your use case? > > Thanks, > John > > On Sun, Jun 28, 2020, at 14:09, Navinder Brar wrote: >> Sorry my

Re: [VOTE] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-07-27 Thread Navinder Brar
+1 (non-binding). Thanks for the KIP, Bruno. ~Navinder On Friday, 24 July, 2020, 08:41:03 pm IST, John Roesler wrote: Thanks, Bruno! I'm +1 (binding) -John On Fri, Jul 24, 2020, at 07:04, Bruno Cadonna wrote: > Hi, > > After re-opening the discussion about > > https://cwiki.apach

Re: [VOTE] KIP-648: Renaming getter method for Interactive Queries

2020-07-29 Thread Navinder Brar
+1 (non-binding). Thanks John, looks good to me. ~NavinderOn Wednesday, 29 July, 2020, 04:32:25 am IST, John Thomas wrote: Hello everyone, I'd like to kick-off a vote for KIP-648 : Renaming getter method for Interactive Queries https://cwiki.apache.org/confluence/display/KAFKA/KIP-648

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-08-02 Thread Navinder Brar
for Guozhang, thanks to Matthias we have fixed it last week as part of KAFKA-10306. ~Navinder On Sunday, 26 July, 2020, 07:37:34 pm IST, Navinder Brar wrote: Hi, Sorry, it took some time to respond back. “but I thought we would pass the config through to the client.” >

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-08-09 Thread Navinder Brar
tstrap the global stores before processing happens. Thus, if an `InvalidOffsetException` happen and the global thread dies, the main threads cannot access the global stores any longer an also die. If we re-build the state though, do we need to pause the main thread during this phase? -Matthias

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-08-16 Thread Navinder Brar
making some progress.  Thanks, John On Sun, Aug 9, 2020, at 02:04, Navinder Brar wrote: > Hi Matthias, > > IMHO, now as you explained using ‘global.consumer.auto.offset.reset’ is > not as straightforward  > as it seems and it might change the existing behavior for users w

Re: [VOTE] KIP-657: Add Customized Kafka Streams Logo

2020-08-18 Thread Navinder Brar
Hi, Thanks for the KIP, really like the idea. I am +1(non-binding) on A mainly because I felt like you have to tilt your head to realize the otter's head in B.  Regards,Navinder On Tuesday, 18 August, 2020, 11:44:20 pm IST, Guozhang Wang wrote: I'm leaning towards design B primarily

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-08-19 Thread Navinder Brar
; -Matthias >> >> >> >> >> On 8/17/20 7:34 AM, John Roesler wrote: >>> Hi Navinder, >>> >>> I see what you mean about the global consumer being similar >>> to the restore consumer. >>> >>> I also agree that aut

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-08-28 Thread Navinder Brar
Gentle ping. ~ Navinder On Wednesday, 19 August, 2020, 06:59:58 pm IST, Navinder Brar wrote: Thanks Matthias & John,  I am glad we are converging towards an understanding. So, to summarize,  we will still keep treating this change in KIP and instead of providing a reset stra

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-09-03 Thread Navinder Brar
gt;> >> >> -Matthias >> >> On 8/29/20 9:37 AM, John Roesler wrote: >>> Hi Navinder, >>> >>> Thanks for the ping. Yes, that all sounds right to me. The name >> “RESTORING_GLOBAL” sounds fine, too. >>> >>> I think as far

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-09-04 Thread Navinder Brar
; > > Do we think this would work? -- Of course, the devil is in the details > but it seems to become a PR discussion, and there is no reason to make > it part of the KIP. > > > -Matthias > > On 9/3/20 3:41 AM, Navinder Brar wrote: > > Hi, > > > > Thanks

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-09-13 Thread Navinder Brar
res that actually need to be restored block anything 2. Only the tasks that access the stores get blocked 3. No new states need to be introduced WDYT? -John On Fri, 2020-09-04 at 13:18 +, Navinder Brar wrote: > Hi Sophie, > > Thanks for the detailed explanation. I agree from a user sta

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-09-22 Thread Navinder Brar
processed in the topology? We'd have to remember which ProcessorNode to resume from when we re-try later. This is really where the spiritual overlap with KIP-572 comes in. Maybe Matthias can share some thoughts. Thanks, -John On Sun, 2020-09-13 at 07:50 +, Navinder Brar wrote: >  &

Re: [ANNOUNCE] Apache Kafka 2.5.0

2020-04-15 Thread Navinder Brar
Thanks for running the release David. Congratulations to everyone involved. -Navinder Pal Singh Brar On Thursday, 16 April, 2020, 07:26:25 am IST, Konstantine Karantasis wrote: Thanks for driving the release David and congrats to all the contributors. Nicely done! Konstantine On We

Re: Potential Bug in 2.3 version (leading to deletion of state directories)

2019-11-14 Thread Navinder Brar
entually is able to re-create the state directory (after the cleaner is done) and transition to RUNNING. This should be the case. If not, I would consider it a bug. Thanks, -John On Thu, Nov 14, 2019 at 3:02 PM Navinder Brar wrote: > > Hi, > We are facing a peculiar situation in the 2.3

Re: [VOTE] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-17 Thread Navinder Brar
3:48 PM, Guozhang Wang wrote: > > > +1 (binding), thanks for the KIP! > > > > > > Guozhang > > > > > > On Fri, Nov 15, 2019 at 4:38 AM Navinder Brar > > > wrote: > > > > > >> Hello all, > > >> > > &g

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2020-01-14 Thread Navinder Brar
+1 (non-binding) Navinder On Tuesday, 14 January, 2020, 07:24:02 pm IST, Ryanne Dolan wrote: Bump. We've got 4 non-binding and one binding vote. Ryanne On Fri, Dec 13, 2019, 1:44 AM Tom Bentley wrote: > +1 (non-binding) > > On Thu, Dec 12, 2019 at 6:33 PM Andrew Schofield < > andrew_

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

2020-01-14 Thread Navinder Brar
+1 (non-binding) With a small comment which was mentioned by Vinoth as well. Did we fix on the flag for StreamsRebalancingException, I don't see it in the KIP. -Navinder On Tuesday, 14 January, 2020, 08:00:11 pm IST, Vito Jeng wrote: Hi, all, I would like to start the vote for KIP-2

Re: [VOTE] KIP-535: Allow state stores to serve stale reads during rebalance

2020-01-15 Thread Navinder Brar
active tasks can be queried during recovery > and > > >>>>> no exception would be thrown and longer. This is a change in > behavior > > >>>>> and in fact introduces a race condition for users that only want to > > >>>>> query con

Role of CompositeReadOnlyKeyValueStore for point queries

2020-01-15 Thread Navinder Brar
Hi all, Can someone explain to me the thoughts behind having CompositeReadOnlyKeyValueStore. java class while serving data via APIs in Kafka Streams. It fetches the list of stores for all the running tasks on the machine and then looks for a key one by one in each of the stores. When we already

Re: Role of CompositeReadOnlyKeyValueStore for point queries

2020-01-16 Thread Navinder Brar
could just replace the new overload instead of deprecating it. But it's not such a huge deal to deprecate it, so no pressure. Thanks for the astute observation! -John On Wed, Jan 15, 2020, at 22:50, Navinder Brar wrote: > Hi all, > Can someone explain to me the thoughts behind having >

[DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-17 Thread Navinder Brar
Hi all, I have created a new KIP:  https://cwiki.apache.org/confluence/display/KAFKA/KIP-562%3A+Allow+fetching+a+key+from+a+single+partition+rather+than+iterating+over+all+the+stores+on+an+instance Please take a look if you get a chance. ~Navinder

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-17 Thread Navinder Brar
I'm stirring the pot, but what do you think about this? Oh, also, the KIP is missing the method signature for the new KafkaStreams#store overload. Thanks! -John On Fri, Jan 17, 2020, at 08:07, Navinder Brar wrote: > Hi all, > I have created a new > KIP:  > https://cwiki.apache.

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-18 Thread Navinder Brar
ate "all", but that seems a little complicated. > > > > > > Thus, maybe it would actually be better to eschew withPartitions for > now > > > and instead just offer: > > > withPartition(int partition) > > > withAllLocalPartitions() > > &

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-19 Thread Navinder Brar
I have made some edits in the KIP, please take another look. It would be great if we can push it in 2.5.0. ~Navinder On Sunday, January 19, 2020, 12:59 AM, Navinder Brar wrote: Sure John, I will update the StoreQueryParams with static factory methods. @Ted, we would need to create taskId

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-20 Thread Navinder Brar
see the public interfaces. We can discuss the implementation in the PR. That said, the public API part of the current proposal looks good to me! I would be a +1 if you called for a vote. Thanks, John On Sun, Jan 19, 2020, at 20:50, Navinder Brar wrote: > I have made some edits in the

[VOTE] KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-20 Thread Navinder Brar
Hello all, I'd like to propose a vote to serve keys from a specific partition-store instead of iterating over all the local stores of an instance to locate the key, as which happens currently. The full KIP is provided here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-562%3A+Allow+fet

Re: [VOTE] KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-21 Thread Navinder Brar
ctors. Guozhang On Mon, Jan 20, 2020 at 10:42 AM Navinder Brar wrote: > Hello all, > > I'd like to propose a vote to serve keys from a specific partition-store > instead of iterating over all the local stores of an instance to locate the > key, as which happens currently. > T

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-22 Thread Navinder Brar
o > filter out based on partitions makes sense > > > > > > > > On Mon, Jan 20, 2020 at 3:13 AM Navinder Brar > wrote: > >> Thanks John. If there are no other comments to be addressed, I will start >> a vote today so that we are on track for this r

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-22 Thread Navinder Brar
, this proposal would be 100% in line with the grammar, and IMO ready to go. Thanks, -John Thanks, -John On Wed, Jan 22, 2020, at 03:56, Navinder Brar wrote: > Thanks Matthias for the feedback. > > 10) As Guozhang suggested above, we thought of adding storeName and > queryableStore

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-22 Thread Navinder Brar
plan is to implement  new APIs following the grammar and gradually to port old APIs to it. The grammar wiki page has plenty of justification, so I won't recapitulate it here. Thanks, -John On Wed, Jan 22, 2020, at 09:39, Navinder Brar wrote: > 10) Sure John, please go ahead. > > 2

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-22 Thread Navinder Brar
we deprecate this method in favor of the new `store(StoreQueryParams)` overload? -Matthias On 1/22/20 10:06 AM, John Roesler wrote: > Thanks Navinder! I've also updated the motivation. > > Thanks, > -John > > On Wed, Jan 22, 2020, at 11:12, Navinder Brar wrote: >

Re: [VOTE] KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-22 Thread Navinder Brar
> > > Thanks, > > -John > > > > On Tue, Jan 21, 2020, at 12:50, Navinder Brar wrote: > >> Thanks, Guozhang. I agree it makes total sense. I will make the > >> edits.~Navinder > >> > >>    On Tuesday, 21 January, 2020, 11:00:32 pm IST,

Re: [VOTE] KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-22 Thread Navinder Brar
Oh sorry, my bad. Will wait for another 12 hours. On Thursday, 23 January, 2020, 12:09:57 pm IST, Matthias J. Sax wrote: Navinder, a KIP vote must be open for 72h and cannot be closed earlier. -Matthias On 1/22/20 10:27 PM, Navinder Brar wrote: > Thanks, everyone for voting.

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-23 Thread Navinder Brar
about the `StoreQueryParams`. I think it should be immutable. That is more an implementation detail and we should discuss it on the PR. Just wanted to mention it in advance. Probably we should add also a rule for immutability to the grammar. Best, Bruno On Wed, Jan 22, 2020 at 7:38 PM Navinder Brar wrote:

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-23 Thread Navinder Brar
uot;, > > etc. > > > > Maybe avoiding discussion was too ambitious, and I can't deny that > > bean spec names probably result in no one being happy, so I'm on > > board with the current proposal: > > > > setters: > > set{FieldName}(value)

Re: [VOTE] KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-23 Thread Navinder Brar
, 2020 at 9:19 AM John Roesler wrote: > > Thanks, Navinder. It's just to give everyone a chance to object if they > wanted to. > -John > > On Thu, Jan 23, 2020, at 00:44, Navinder Brar wrote: > > Oh sorry, my bad. Will wait for another 12 hours. > > > >    On Th

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

2020-02-04 Thread Navinder Brar
Thanks Vito, for incorporating this. Makes sense. -Navinder On Wednesday, February 5, 2020, 12:17 AM, Matthias J. Sax 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) alrea

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-02-06 Thread Navinder Brar
Hi, While implementing 562, we have decided to rename StoreQueryParams -> StoreQueryParameters. I have updated the PR and confluence. Please share if anyone has feedback on it. Thanks & Regards, Navinder Pal Singh Brar On Friday, 24 January, 2020, 08:45:15 am IST, Navinder Brar

Re: [VOTE] KIP-568: Explicit rebalance triggering on the Consumer

2020-02-11 Thread Navinder Brar
Thanks Sophie, much required. +1 non-binding. Sent from Yahoo Mail for iPhone On Tuesday, February 11, 2020, 10:33 PM, John Roesler wrote: Thanks Sophie, I'm +1 (binding) -John On Mon, Feb 10, 2020, at 20:54, Sophie Blee-Goldman wrote: > Hey all, > > I'd like to start the voting on KIP-5

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

2020-02-21 Thread Navinder Brar
/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 wrote: > Thanks Vito, for incorporating t

Re: [ANNOUNCE] New Kafka PMC Member: Bill Bejeck

2021-04-21 Thread Navinder Brar
Congrats Bill, very well deserved! Regards,Navinder Pal Singh BrarLinkedIn On Wednesday, 21 April, 2021, 10:37:53 pm IST, Sarwar Bhuiyan wrote: Congrats Bill! On Wed, Apr 21, 2021 at 5:52 PM Mickael Maison wrote: > Congratulations Bill, well deserved! > > On Sat, Apr 17, 2021 at 1

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-10-03 Thread Navinder Brar
change it. -Matthias On 9/22/20 11:08 PM, Navinder Brar wrote: > Thanks a lot John for these suggestions. @Matthias can share your thoughts on > the last two comments made in this chain. > > Thanks,Navinder > >    On Monday, 14 September, 2020, 09:03:32 pm IST, Jo

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-10-08 Thread Navinder Brar
IQ >> for the duration of the rebalance. But these days, the threads can continue >> processing >> any tasks they own during a rebalance, so the only time that tasks are >> truly unavailable >> is during the restoration phase. >> >> So, I find the idea of ge

Re: [ANNOUNCE] New committer: David Jacot

2020-10-19 Thread Navinder Brar
Many Congratulations David. Best Regards,Navinder On Monday, 19 October, 2020, 12:53:43 pm IST, Dongjin Lee wrote: Congratulations, David! Best, Dongjin On Mon, Oct 19, 2020 at 12:20 PM Hu Xi wrote: > Congrats, David! Well deserved! > > > > 发件人: Vah

Re: [ANNOUNCE] New committer: A. Sophie Blee-Goldman

2020-10-19 Thread Navinder Brar
That's great news. Congrats Sophie! Well deserved. Regards, Navinder On Monday, 19 October, 2020, 10:12:16 pm IST, Bruno Cadonna wrote: Congrats Sophie! Very well deserved! Bruno On 19.10.20 18:40, Matthias J. Sax wrote: > Hi all, > > I am excited to announce that A. Sophie Blee-Gold

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-10-26 Thread Navinder Brar
, Navinder On Thursday, 8 October, 2020, 11:31:28 pm IST, Navinder Brar wrote: Thanks, Sophie, Guozhang, and Matthias for sharing your thoughts. I am glad that another meaningful KIP is coming out of this discussion. I am good towards parking this KIP, till we can make the changes

[jira] [Created] (KAFKA-6924) Making state store queryable on replicas

2018-05-20 Thread Navinder Brar (JIRA)
Navinder Brar created KAFKA-6924: Summary: Making state store queryable on replicas Key: KAFKA-6924 URL: https://issues.apache.org/jira/browse/KAFKA-6924 Project: Kafka Issue Type: New

[jira] [Created] (KAFKA-6643) Warm up new replicas from scratch when changelog topic has retention time

2018-03-12 Thread Navinder Brar (JIRA)
Navinder Brar created KAFKA-6643: Summary: Warm up new replicas from scratch when changelog topic has retention time Key: KAFKA-6643 URL: https://issues.apache.org/jira/browse/KAFKA-6643 Project

[jira] [Created] (KAFKA-9169) Standby Tasks point ask for incorrect offsets on resuming post suspension

2019-11-10 Thread Navinder Brar (Jira)
Navinder Brar created KAFKA-9169: Summary: Standby Tasks point ask for incorrect offsets on resuming post suspension Key: KAFKA-9169 URL: https://issues.apache.org/jira/browse/KAFKA-9169 Project

[jira] [Created] (KAFKA-10429) Group Coordinator is unavailable leads to missing events

2020-08-24 Thread Navinder Brar (Jira)
Navinder Brar created KAFKA-10429: - Summary: Group Coordinator is unavailable leads to missing events Key: KAFKA-10429 URL: https://issues.apache.org/jira/browse/KAFKA-10429 Project: Kafka

[jira] [Created] (KAFKA-9445) Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-16 Thread Navinder Brar (Jira)
Navinder Brar created KAFKA-9445: Summary: Allow fetching a key from a single partition rather than iterating over all the stores on an instance Key: KAFKA-9445 URL: https://issues.apache.org/jira/browse/KAFKA

[jira] [Created] (KAFKA-9487) Followup : KAFKA-9445

2020-01-31 Thread Navinder Brar (Jira)
Navinder Brar created KAFKA-9487: Summary: Followup : KAFKA-9445 Key: KAFKA-9487 URL: https://issues.apache.org/jira/browse/KAFKA-9487 Project: Kafka Issue Type: New Feature

[jira] [Resolved] (KAFKA-9487) Followup : KAFKA-9445(Allow fetching a key from a single partition); addressing code review comments

2020-02-11 Thread Navinder Brar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9487?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Navinder Brar resolved KAFKA-9487. -- Resolution: Fixed > Followup : KAFKA-9445(Allow fetching a key from a single partit

[jira] [Created] (KAFKA-9588) Add rocksdb event listeners in KS

2020-02-20 Thread Navinder Brar (Jira)
Navinder Brar created KAFKA-9588: Summary: Add rocksdb event listeners in KS Key: KAFKA-9588 URL: https://issues.apache.org/jira/browse/KAFKA-9588 Project: Kafka Issue Type: New Feature