Re: [VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-11-03 Thread Anna Povzner
in the bucket, in addition to the current rate > metric. It would be great to add it in the metrics section of the KIP as > well > for completeness. > > Best, > David > > On Tue, Aug 11, 2020 at 4:28 AM Anna Povzner wrote: > > > Hi All, > > > >

Re: [DISCUSS] Apache Kafka 2.7.0 release

2020-10-06 Thread Anna Povzner
Hi Bill, Regarding KIP-612, only the first half of the KIP will get into 2.7 release: Broker-wide and per-listener connection rate limits, including corresponding configs and metric (KAFKA-10023). I see that the table in the release plan tags KAFKA-10023 as "old", not sure what it refers to. Note

[jira] [Created] (KAFKA-10509) Add metric to track throttle time due to hitting connection rate quota

2020-09-21 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-10509: Summary: Add metric to track throttle time due to hitting connection rate quota Key: KAFKA-10509 URL: https://issues.apache.org/jira/browse/KAFKA-10509 Project

[jira] [Resolved] (KAFKA-10458) Need a way to update quota for TokenBucket registered with Sensor

2020-09-21 Thread Anna Povzner (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner resolved KAFKA-10458. -- Resolution: Fixed > Need a way to update quota for TokenBucket registered with Sen

[jira] [Created] (KAFKA-10458) Need a way to update quota for TokenBucket registered with Sensor

2020-09-02 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-10458: Summary: Need a way to update quota for TokenBucket registered with Sensor Key: KAFKA-10458 URL: https://issues.apache.org/jira/browse/KAFKA-10458 Project: Kafka

Re: [VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-08-10 Thread Anna Povzner
have any concerns. Thanks, Anna On Thu, May 21, 2020 at 5:42 PM Anna Povzner wrote: > The vote for KIP-612 has passed with 3 binding and 3 non-binding +1s, and > no objections. > > > Thanks everyone for reviews and feedback, > > Anna > > On Tue, May 19, 2020 at 2:41

[jira] [Created] (KAFKA-10157) Multiple tests failed due to "Failed to process feature ZK node change event"

2020-06-11 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-10157: Summary: Multiple tests failed due to "Failed to process feature ZK node change event" Key: KAFKA-10157 URL: https://issues.apache.org/jira/browse/K

Re: [VOTE] KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-06-05 Thread Anna Povzner
+1 (not binding) Thanks for the KIP! -Anna On Thu, Jun 4, 2020 at 8:26 AM Mickael Maison wrote: > +1 (binding) > Thanks David for looking into this important issue > > On Thu, Jun 4, 2020 at 3:59 PM Tom Bentley wrote: > > > > +1 (non binding). > > > > Thanks! > > > > On Wed, Jun 3, 2020 at 3:

Re: KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-06-05 Thread Anna Povzner
few insights. > > I don't have specific numbers that can be used out of the box so I > > think that it is better to not put any for the time being. We can always > > complement later on in the documentation. > > > > Please, take a look and let me know what you thi

Re: KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-06-04 Thread Anna Povzner
> > > > > > > > > Hi Jun and Anna, > > > > > > > > > > Thank you both for your replies. > > > > > > > > > > Based on our recent discussion, I agree that using a rate is better > > to > > > > > remain

Re: KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-06-03 Thread Anna Povzner
Hi Jun and David, Regarding token bucket vs, Rate behavior. We recently observed a couple of cases where a bursty workload behavior would result in long-ish pauses in between, resulting in lower overall bandwidth than the quota. I will need to debug this a bit more to be 100% sure, but it does loo

Re: [VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-21 Thread Anna Povzner
the nice feature to control the connection creation > > rate > > > > from the clients. > > > > > > > > On Tue, May 19, 2020 at 8:16 AM Gwen Shapira > > wrote: > > > > > > > > > +1 (binding) > > > > > > &

[jira] [Created] (KAFKA-10024) Add dynamic configuration and enforce quota for per-IP connection rate limits

2020-05-19 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-10024: Summary: Add dynamic configuration and enforce quota for per-IP connection rate limits Key: KAFKA-10024 URL: https://issues.apache.org/jira/browse/KAFKA-10024

[jira] [Created] (KAFKA-10023) Enforce broker-wide and per-listener connection creation rate (KIP-612, part 1)

2020-05-19 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-10023: Summary: Enforce broker-wide and per-listener connection creation rate (KIP-612, part 1) Key: KAFKA-10023 URL: https://issues.apache.org/jira/browse/KAFKA-10023

[VOTE] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
Hi All, I would like to start the vote on KIP-612: Ability to limit connection creation rate on brokers. For reference, here is the KIP wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-612%3A+Ability+to+Limit+Connection+Creation+Rate+on+Brokers And discussion thread: https://lists.apa

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
I realized the KIP freeze is on May 20. I will start the voting thread now. On Mon, May 18, 2020 at 3:19 PM Anna Povzner wrote: > Thanks everyone for the feedback. I will start a voting thread tomorrow > morning if there are no more comments. > > Regards, > Anna > > On Mon,

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
Thanks everyone for the feedback. I will start a voting thread tomorrow morning if there are no more comments. Regards, Anna On Mon, May 18, 2020 at 2:06 PM Anna Povzner wrote: > Hi Boyang, > > This KIP does not change the protocol with clients. The behavior is the > same as with K

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
Just want to clarify what's the expected behavior to be seen on the > client side? > > On Mon, May 18, 2020 at 12:04 PM Anna Povzner wrote: > > > Hi Alexandre, > > > > Thanks for your comments. My answers are below: > > > > 900. The KIP does not propose

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-18 Thread Anna Povzner
/tree/trunk/tools/src/main/java/org/apache/kafka/trogdor/workload Regards, Anna On Mon, May 18, 2020 at 8:57 AM Rajini Sivaram wrote: > Hi Anna, > > Thanks for the response, sounds good. > > Regards, > > Rajini > > > On Sun, May 17, 2020 at 1:38 AM Anna P

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-16 Thread Anna Povzner
> dropping doesn't really help since that IP is going to reconnect. If we > > want to rate limit what happens next, i.e. authentication, then > > throttling the accepted connection so its processing is delayed would > > perhaps be better? > > > > Regards, > &g

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-13 Thread Anna Povzner
them until the rate is back within the rate limit. On Wed, May 13, 2020 at 11:46 AM Anna Povzner wrote: > Hi David and Alexandre, > > Thanks so much for your feedback! Here are my answers: > > 1. Yes, we have seen several cases of clients that create a new connection > per produ

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-13 Thread Anna Povzner
the number of connections per IP? Instead of rate limiting all the > > creation, > > we could perhaps rate limit the number of creation per IP as well. That > > could > > perhaps reduce the effect on the other clients. That may be harder to > > implement > >

[DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-11 Thread Anna Povzner
Hi, I just created KIP-612 to allow limiting connection creation rate on brokers, and would like to start a discussion. https://cwiki.apache.org/confluence/display/KAFKA/KIP-612%3A+Ability+to+Limit+Connection+Creation+Rate+on+Brokers Feedback and suggestions are welcome! Thanks, Anna

Re: KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-05-06 Thread Anna Povzner
Hi David and Jun, I wanted to add to the discussion about using requests/sec vs. time on server threads (similar to request quota) for expressing quota for topic ops. I think request quota does not protect the brokers from overload by itself -- it still requires tuning and sometimes re-tuning, be

[jira] [Created] (KAFKA-9839) IllegalStateException on metadata update when broker learns about its new epoch after the controller

2020-04-08 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-9839: --- Summary: IllegalStateException on metadata update when broker learns about its new epoch after the controller Key: KAFKA-9839 URL: https://issues.apache.org/jira/browse/KAFKA-9839

[jira] [Created] (KAFKA-9677) Low consume bandwidth quota may cause consumer not being able to fetch data

2020-03-06 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-9677: --- Summary: Low consume bandwidth quota may cause consumer not being able to fetch data Key: KAFKA-9677 URL: https://issues.apache.org/jira/browse/KAFKA-9677 Project

[jira] [Created] (KAFKA-9658) Removing default user quota doesn't take effect until broker restart

2020-03-04 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-9658: --- Summary: Removing default user quota doesn't take effect until broker restart Key: KAFKA-9658 URL: https://issues.apache.org/jira/browse/KAFKA-9658 Project:

Re: [DISCUSS] KIP-546: Add quota-specific APIs to the Admin Client, redux

2020-01-24 Thread Anna Povzner
Hi Brian, The KIP looks good! I have one clarification question regarding the distinction between describe and resolve API. Suppose I set request quota for /config/users/”user-1”/clients/"client-1" to 100 and request quota for /config/users/”user-1” to 200. Is this correct that describeClientQuot

[jira] [Reopened] (KAFKA-8800) Flaky Test SaslScramSslEndToEndAuthorizationTest#testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl

2019-08-26 Thread Anna Povzner (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8800?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner reopened KAFKA-8800: - Assignee: Anastasia Vela (was: Lee Dongjin) > Flaky T

[jira] [Created] (KAFKA-8837) KafkaMetricReporterClusterIdTest may not shutdown ZooKeeperTestHarness

2019-08-26 Thread Anna Povzner (Jira)
Anna Povzner created KAFKA-8837: --- Summary: KafkaMetricReporterClusterIdTest may not shutdown ZooKeeperTestHarness Key: KAFKA-8837 URL: https://issues.apache.org/jira/browse/KAFKA-8837 Project: Kafka

[jira] [Created] (KAFKA-8782) ReplicationQuotaManagerTest and ClientQuotaManagerTest should close Metrics object

2019-08-09 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-8782: --- Summary: ReplicationQuotaManagerTest and ClientQuotaManagerTest should close Metrics object Key: KAFKA-8782 URL: https://issues.apache.org/jira/browse/KAFKA-8782

[jira] [Created] (KAFKA-8526) Broker may select a failed dir for new replica even in the presence of other live dirs

2019-06-11 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-8526: --- Summary: Broker may select a failed dir for new replica even in the presence of other live dirs Key: KAFKA-8526 URL: https://issues.apache.org/jira/browse/KAFKA-8526

[jira] [Created] (KAFKA-8481) Clients may fetch incomplete set of topic partitions just after topic is created

2019-06-04 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-8481: --- Summary: Clients may fetch incomplete set of topic partitions just after topic is created Key: KAFKA-8481 URL: https://issues.apache.org/jira/browse/KAFKA-8481 Project

[jira] [Created] (KAFKA-8480) Clients may fetch incomplete set of topic partitions during cluster startup

2019-06-04 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-8480: --- Summary: Clients may fetch incomplete set of topic partitions during cluster startup Key: KAFKA-8480 URL: https://issues.apache.org/jira/browse/KAFKA-8480 Project

[jira] [Created] (KAFKA-8002) Replica reassignment to new log dir may not complete if future and current replicas segment files have different base offsets

2019-02-25 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-8002: --- Summary: Replica reassignment to new log dir may not complete if future and current replicas segment files have different base offsets Key: KAFKA-8002 URL: https

[jira] [Created] (KAFKA-8001) Fetch from future replica stalls when local replica becomes a leader

2019-02-25 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-8001: --- Summary: Fetch from future replica stalls when local replica becomes a leader Key: KAFKA-8001 URL: https://issues.apache.org/jira/browse/KAFKA-8001 Project: Kafka

[jira] [Created] (KAFKA-7923) Add unit test to verify Kafka-7401 in AK versions >= 2.0

2019-02-12 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-7923: --- Summary: Add unit test to verify Kafka-7401 in AK versions >= 2.0 Key: KAFKA-7923 URL: https://issues.apache.org/jira/browse/KAFKA-7923 Project: Kafka Is

[jira] [Created] (KAFKA-7786) Fast update of leader epoch may stall partition fetching due to FENCED_LEADER_EPOCH

2019-01-03 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-7786: --- Summary: Fast update of leader epoch may stall partition fetching due to FENCED_LEADER_EPOCH Key: KAFKA-7786 URL: https://issues.apache.org/jira/browse/KAFKA-7786

[jira] [Created] (KAFKA-7415) OffsetsForLeaderEpoch may incorrectly respond with undefined epoch causing truncation to HW

2018-09-14 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-7415: --- Summary: OffsetsForLeaderEpoch may incorrectly respond with undefined epoch causing truncation to HW Key: KAFKA-7415 URL: https://issues.apache.org/jira/browse/KAFKA-7415

Re: [VOTE] KIP-320: Allow fetchers to detect and handle log truncation

2018-08-13 Thread Anna Povzner
+1 Thanks for the KIP! On Thu, Aug 9, 2018 at 5:16 PM Jun Rao wrote: > Hi, Jason, > > Thanks for the KIP. +1 from me. > > Jun > > On Wed, Aug 8, 2018 at 1:04 PM, Jason Gustafson > wrote: > > > Hi All, > > > > I'd like to start a vote for KIP-320: > > https://cwiki.apache.org/confluence/display

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-26 Thread Anna Povzner
%3A+Allow+fetchers+to+detect+and+handle+log+truncation) > > with a section say "workflow" on how the proposed API will be co-used > with > > others to: > > > > 1. consumer callers handling a LogTruncationException. > > 2. consumer internals for handling a

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-17 Thread Anna Povzner
> > problems, but I am still verifying the model. > > > > Thanks, > > Jason > > > > > > On Wed, Jul 11, 2018 at 10:45 AM, Dong Lin wrote: > > > > > Hey Anna, > > > > > > Thanks much for the explanation. Approach 1 also sound

[jira] [Created] (KAFKA-7151) Broker running out of disk space may result in state where unclean leader election is required

2018-07-11 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-7151: --- Summary: Broker running out of disk space may result in state where unclean leader election is required Key: KAFKA-7151 URL: https://issues.apache.org/jira/browse/KAFKA-7151

[jira] [Created] (KAFKA-7150) Error in processing fetched data for one partition may stop follower fetching other partitions

2018-07-11 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-7150: --- Summary: Error in processing fetched data for one partition may stop follower fetching other partitions Key: KAFKA-7150 URL: https://issues.apache.org/jira/browse/KAFKA-7150

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-11 Thread Anna Povzner
hand, if we use seek(offset), consumer will > not detect log truncation in some cases which weakens the guarantee of this > KIP. Does this make sense? > > Thanks, > Dong > > On Tue, Jul 10, 2018 at 3:14 PM, Anna Povzner wrote: > > > Sorry, I hit "send" befo

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-10 Thread Anna Povzner
n that the consumer will not be able to detect log truncation in all cases? Thanks, Anna On Tue, Jul 10, 2018 at 2:20 PM Anna Povzner wrote: > Hi Dong, > > Thanks for the follow up! I finally have much more clear understanding of > where you are coming from. > > You are right

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-10 Thread Anna Povzner
...), then user can probably just call > Consumer#seekToEnd() without calling position() and seek(...). Similarly > user can call Consumer#seekToBeginning() to the seek to the earliest > position without calling position() and seek(...). Thus position() only > needs to return the actual consumed

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-09 Thread Anna Povzner
ot; use-case, it seems that there can be two > positions, i.e. earliest and latest. So these two cases can be > Consumer.fulfilled by seekToBeginning() and Consumer.seekToEnd(). Then it > seems that user will only need to call position() and seek() for "seeking > to a message"

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-07-04 Thread Anna Povzner
Hi Jason and Dong, I’ve been thinking about your suggestions and discussion regarding position(), seek(), and new proposed API. Here is my thought process why we should keep position() and seek() API unchanged. I think we should separate {offset, leader epoch} that uniquely identifies a messa

[jira] [Created] (KAFKA-7104) ReplicaFetcher thread may die because of inconsistent log start offset in fetch response

2018-06-26 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-7104: --- Summary: ReplicaFetcher thread may die because of inconsistent log start offset in fetch response Key: KAFKA-7104 URL: https://issues.apache.org/jira/browse/KAFKA-7104

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2018-06-14 Thread Anna Povzner
Hi Tom, Just wanted to check what you think about the comments I made in my last message. I think this KIP is a big improvement to our current policy interfaces, and really hope we can get this KIP in. Thanks, Anna On Thu, May 31, 2018 at 3:29 PM, Anna Povzner wrote: > Hi Tom, > >

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2018-05-31 Thread Anna Povzner
Hi Tom, Thanks for the KIP. I am aware that the voting thread was started, but wanted to discuss couple of concerns here first. I think the coupling of RequestedTopicState#generatedReplicaAssignment() and TopicState#replicasAssignments() does not work well in case where the request deals only w

[jira] [Created] (KAFKA-6975) AdminClient.deleteRecords() may cause replicas unable to fetch from beginning

2018-05-31 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6975: --- Summary: AdminClient.deleteRecords() may cause replicas unable to fetch from beginning Key: KAFKA-6975 URL: https://issues.apache.org/jira/browse/KAFKA-6975 Project

[jira] [Resolved] (KAFKA-6795) Add unit test for ReplicaAlterLogDirsThread

2018-05-04 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6795?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner resolved KAFKA-6795. - Resolution: Fixed Fix Version/s: 2.0.0 > Add unit test for ReplicaAlterLogDirsThr

[jira] [Created] (KAFKA-6859) Follower should not send OffsetForLeaderEpoch for undefined leader epochs

2018-05-03 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6859: --- Summary: Follower should not send OffsetForLeaderEpoch for undefined leader epochs Key: KAFKA-6859 URL: https://issues.apache.org/jira/browse/KAFKA-6859 Project: Kafka

[jira] [Created] (KAFKA-6824) Transient failure in DynamicBrokerReconfigurationTest.testAddRemoveSslListener

2018-04-24 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6824: --- Summary: Transient failure in DynamicBrokerReconfigurationTest.testAddRemoveSslListener Key: KAFKA-6824 URL: https://issues.apache.org/jira/browse/KAFKA-6824 Project

[jira] [Created] (KAFKA-6823) Transient failure in DynamicBrokerReconfigurationTest.testThreadPoolResize

2018-04-24 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6823: --- Summary: Transient failure in DynamicBrokerReconfigurationTest.testThreadPoolResize Key: KAFKA-6823 URL: https://issues.apache.org/jira/browse/KAFKA-6823 Project

[jira] [Created] (KAFKA-6809) connections-created metric does not behave as expected

2018-04-19 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6809: --- Summary: connections-created metric does not behave as expected Key: KAFKA-6809 URL: https://issues.apache.org/jira/browse/KAFKA-6809 Project: Kafka Issue

Re: [VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-17 Thread Anna Povzner
The vote for KIP-279 has passed with 5 binding and 2 non-binding +1s (and no objections). Thanks everyone for your reviews and feedback, Anna On Tue, Apr 17, 2018 at 1:49 PM, Anna Povzner wrote: > Guozhang, thanks for catching this, I fixed the description (the example > assumed re

Re: [VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-17 Thread Anna Povzner
Guozhang, thanks for catching this, I fixed the description (the example assumed response with 21, '11' was a typo). On Tue, Apr 17, 2018 at 1:25 PM, Anna Povzner wrote: > Hi Colin, > > Yes, the impact of "losing" entries in the LeaderEpoch file is more > roun

Re: [VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-17 Thread Anna Povzner
g a leader change. That's the > only impact, right? > > best, > Colin > > On Mon, Apr 16, 2018, at 09:48, Ismael Juma wrote: > > Thanks for the detailed KIP. +1 (binding) > > > > Ismael > > > > On Sat, Apr 14, 2018 at 3:54 PM, Anna Povzner wrote: >

[jira] [Created] (KAFKA-6795) Add unit test for ReplicaAlterLogDirsThread

2018-04-16 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6795: --- Summary: Add unit test for ReplicaAlterLogDirsThread Key: KAFKA-6795 URL: https://issues.apache.org/jira/browse/KAFKA-6795 Project: Kafka Issue Type

[VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-14 Thread Anna Povzner
Hi All, I would like to start the vote on KIP-279: Fix log divergence between leader and follower after fast leader fail over. For reference, here's the KIP wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+o

Re: [DISCUSS] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-13 Thread Anna Povzner
in a compacted topic by high > watermark. Filed https://issues.apache.org/jira/browse/KAFKA-6780 to track > it. > > Jun > > > On Thu, Apr 5, 2018 at 12:17 PM, Anna Povzner wrote: > > > Hi, > > > > > > I just created KIP-279 to fix edge cases of log di

[jira] [Resolved] (KAFKA-6693) Add Consumer-only benchmark workload to Trogdor

2018-04-10 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner resolved KAFKA-6693. - Resolution: Fixed https://github.com/apache/kafka/pull/4775 > Add Consumer-only benchm

Re: [DISCUSS] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-09 Thread Anna Povzner
for convergence, far > and away the common case would only require a very small number (usually > exactly one). > > Is that correct? > > Thanks, > Jason > > On Fri, Apr 6, 2018 at 5:47 PM, Ted Yu wrote: > > > Makes sense. > > Thanks for the explanation. &

Re: [DISCUSS] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-06 Thread Anna Povzner
> Though the size increases, the number of roundtrips is reduced meaningfully > which would increase the robustness of the solution. > > Please expand the reasoning for unclean leader election for this > alternative. > > Thanks > > On Thu, Apr 5, 2018 at 12:17 PM, Anna

[DISCUSS] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-05 Thread Anna Povzner
Hi, I just created KIP-279 to fix edge cases of log divergence for both clean and unclean leader election configs. https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+over The KIP is basically a follow up to KIP-10

[jira] [Created] (KAFKA-6693) Add Consumer-only benchmark workload to Trogdor

2018-03-20 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6693: --- Summary: Add Consumer-only benchmark workload to Trogdor Key: KAFKA-6693 URL: https://issues.apache.org/jira/browse/KAFKA-6693 Project: Kafka Issue Type

[jira] [Commented] (KAFKA-4691) ProducerInterceptor.onSend() is called after key and value are serialized

2017-01-25 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15838829#comment-15838829 ] Anna Povzner commented on KAFKA-4691: - I agree with [~mjsax] about not chan

[jira] [Work started] (KAFKA-3777) Extract the existing LRU cache out of RocksDBStore

2016-07-22 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3777?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3777 started by Anna Povzner. --- > Extract the existing LRU cache out of RocksDBSt

[jira] [Updated] (KAFKA-3597) Enable query ConsoleConsumer and VerifiableProducer if they shutdown cleanly

2016-04-28 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3597?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-3597: Status: Patch Available (was: In Progress) > Enable query ConsoleConsumer and VerifiableProdu

[jira] [Work started] (KAFKA-3566) Enable VerifiableProducer and ConsoleConsumer to run with interceptors

2016-04-27 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3566 started by Anna Povzner. --- > Enable VerifiableProducer and ConsoleConsumer to run with intercept

[jira] [Work started] (KAFKA-3597) Enable query ConsoleConsumer and VerifiableProducer if they shutdown cleanly

2016-04-27 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3597?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3597 started by Anna Povzner. --- > Enable query ConsoleConsumer and VerifiableProducer if they shutdown clea

[jira] [Created] (KAFKA-3597) Enable query ConsoleConsumer and VerifiableProducer if they shutdown cleanly

2016-04-20 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-3597: --- Summary: Enable query ConsoleConsumer and VerifiableProducer if they shutdown cleanly Key: KAFKA-3597 URL: https://issues.apache.org/jira/browse/KAFKA-3597 Project

[jira] [Created] (KAFKA-3566) Enable VerifiableProducer and ConsoleConsumer to run with interceptors

2016-04-15 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-3566: --- Summary: Enable VerifiableProducer and ConsoleConsumer to run with interceptors Key: KAFKA-3566 URL: https://issues.apache.org/jira/browse/KAFKA-3566 Project: Kafka

[jira] [Created] (KAFKA-3555) Unexpected close of KStreams transformer

2016-04-13 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-3555: --- Summary: Unexpected close of KStreams transformer Key: KAFKA-3555 URL: https://issues.apache.org/jira/browse/KAFKA-3555 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3320) Add successful acks verification to ProduceConsumeValidateTest

2016-03-27 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15213677#comment-15213677 ] Anna Povzner commented on KAFKA-3320: - If you look at verifiable_producer.py

[jira] [Commented] (KAFKA-3202) Add system test for KIP-31 and KIP-32 - Change message format version on the fly

2016-03-10 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15189917#comment-15189917 ] Anna Povzner commented on KAFKA-3202: - [~enothereska] I think you meant to post

[jira] [Updated] (KAFKA-3303) Pass partial record metadata to Interceptor onAcknowledgement in case of errors

2016-03-04 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-3303: Status: Patch Available (was: In Progress) > Pass partial record metadata to Intercep

[jira] [Updated] (KAFKA-3201) Add system test for KIP-31 and KIP-32 - Upgrade Test

2016-03-03 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-3201: Status: Patch Available (was: In Progress) Verified that upgrade tests are passing on Jenkins (ran

[jira] [Work started] (KAFKA-3303) Pass partial record metadata to Interceptor onAcknowledgement in case of errors

2016-03-03 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3303 started by Anna Povzner. --- > Pass partial record metadata to Interceptor onAcknowledgement in case

[jira] [Created] (KAFKA-3320) Add successful acks verification to ProduceConsumeValidateTest

2016-03-02 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-3320: --- Summary: Add successful acks verification to ProduceConsumeValidateTest Key: KAFKA-3320 URL: https://issues.apache.org/jira/browse/KAFKA-3320 Project: Kafka

[jira] [Created] (KAFKA-3303) Pass partial record metadata to Interceptor onAcknowledgement in case of errors

2016-02-29 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-3303: --- Summary: Pass partial record metadata to Interceptor onAcknowledgement in case of errors Key: KAFKA-3303 URL: https://issues.apache.org/jira/browse/KAFKA-3303 Project

[jira] [Updated] (KAFKA-3196) KIP-42 (part 2): add record size and CRC to RecordMetadata and ConsumerRecords

2016-02-24 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3196?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-3196: Status: Patch Available (was: In Progress) > KIP-42 (part 2): add record size and CRC

[jira] [Updated] (KAFKA-3196) KIP-42 (part 2): add record size and CRC to RecordMetadata and ConsumerRecords

2016-02-24 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3196?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-3196: Fix Version/s: 0.10.0.0 > KIP-42 (part 2): add record size and CRC to RecordMetadata

[jira] [Updated] (KAFKA-3214) Add consumer system tests for compressed topics

2016-02-24 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-3214: Fix Version/s: 0.10.0.0 > Add consumer system tests for compressed top

[jira] [Updated] (KAFKA-3214) Add consumer system tests for compressed topics

2016-02-24 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-3214: Status: Patch Available (was: In Progress) > Add consumer system tests for compressed top

[jira] [Work started] (KAFKA-3214) Add consumer system tests for compressed topics

2016-02-23 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3214 started by Anna Povzner. --- > Add consumer system tests for compressed top

[jira] [Commented] (KAFKA-3201) Add system test for KIP-31 and KIP-32 - Upgrade Test

2016-02-23 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15159416#comment-15159416 ] Anna Povzner commented on KAFKA-3201: - Here is a set of tests: 1. Setup: Prod

[jira] [Work started] (KAFKA-3201) Add system test for KIP-31 and KIP-32 - Upgrade Test

2016-02-22 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3201 started by Anna Povzner. --- > Add system test for KIP-31 and KIP-32 - Upgrade T

[jira] [Commented] (KAFKA-3256) Large number of system test failures

2016-02-22 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15157723#comment-15157723 ] Anna Povzner commented on KAFKA-3256: - [~becket_qin] I wrote my comment wit

[jira] [Commented] (KAFKA-3256) Large number of system test failures

2016-02-22 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15157719#comment-15157719 ] Anna Povzner commented on KAFKA-3256: - FYI: The upgrade test fails with this e

[jira] [Commented] (KAFKA-3256) Large number of system test failures

2016-02-22 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15157604#comment-15157604 ] Anna Povzner commented on KAFKA-3256: - [~becket_qin], [~ijuma], [~geoffra]

[jira] [Work started] (KAFKA-3196) KIP-42 (part 2): add record size and CRC to RecordMetadata and ConsumerRecords

2016-02-21 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3196?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3196 started by Anna Povzner. --- > KIP-42 (part 2): add record size and CRC to RecordMetadata and ConsumerReco

[jira] [Commented] (KAFKA-3256) Large number of system test failures

2016-02-20 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15155911#comment-15155911 ] Anna Povzner commented on KAFKA-3256: - Also for completeness, the remaining sy

[jira] [Comment Edited] (KAFKA-3256) Large number of system test failures

2016-02-20 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15155910#comment-15155910 ] Anna Povzner edited comment on KAFKA-3256 at 2/21/16 6:1

[jira] [Commented] (KAFKA-3256) Large number of system test failures

2016-02-20 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15155910#comment-15155910 ] Anna Povzner commented on KAFKA-3256: - I looked more into system tests to find w

[jira] [Comment Edited] (KAFKA-3256) Large number of system test failures

2016-02-20 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15155841#comment-15155841 ] Anna Povzner edited comment on KAFKA-3256 at 2/21/16 1:4

[jira] [Commented] (KAFKA-3256) Large number of system test failures

2016-02-20 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15155841#comment-15155841 ] Anna Povzner commented on KAFKA-3256: - [~becket_qin] The "No JSON object

  1   2   >