Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests are not processed in time

2021-06-26 Thread Dhruvil Shah
Thanks for the KIP, Satish. I am trying to understand the problem we are looking to solve with this KIP. When the leader is slow in processing fetch requests from the follower (due to disk, GC, or other reasons), the primary problem is that it could impact read and write latency and at times

Re: [ANNOUNCE] New Kafka PMC Member: Konstantine Karantasis

2021-06-21 Thread Dhruvil Shah
Congratulations Konstantine! Well deserved! On Mon, Jun 21, 2021 at 10:20 AM Boyang Chen wrote: > Congratulations Konstantine! > > On Mon, Jun 21, 2021 at 10:16 AM Matthias J. Sax wrote: > > > Congrats! > > > > On 6/21/21 12:57 PM, Raymond Ng wrote: > > > Congrats Konstantine! > > > > > > /Ray

[jira] [Created] (KAFKA-12520) Producer state is needlessly rebuilt on startup

2021-03-22 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-12520: Summary: Producer state is needlessly rebuilt on startup Key: KAFKA-12520 URL: https://issues.apache.org/jira/browse/KAFKA-12520 Project: Kafka Issue Type

Re: [DISCUSS] Apache Kafka 2.8.0 release

2021-03-02 Thread Dhruvil Shah
e go ahead and cherry-pick your fix onto the 2.8 > branch. > > Thanks! > -John > > On Mon, 2021-03-01 at 09:36 -0800, Dhruvil Shah wrote: > > Hi John, > > > > I would like to bring up > https://issues.apache.org/jira/browse/KAFKA-12254 > > as a blocker

Re: [DISCUSS] Apache Kafka 2.8.0 release

2021-03-01 Thread Dhruvil Shah
Hi John, I would like to bring up https://issues.apache.org/jira/browse/KAFKA-12254 as a blocker candidate for 2.8.0. While this is not a regression, the issue could lead to data loss in certain cases. The fix is trivial so it may be worth bringing it into 2.8.0. Let me know what you think. -

[jira] [Created] (KAFKA-12254) MirrorMaker 2.0 creates destination topic with default configs

2021-01-29 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-12254: Summary: MirrorMaker 2.0 creates destination topic with default configs Key: KAFKA-12254 URL: https://issues.apache.org/jira/browse/KAFKA-12254 Project: Kafka

[jira] [Created] (KAFKA-10518) Consumer fetches could be inefficient when lags are unbalanced

2020-09-23 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-10518: Summary: Consumer fetches could be inefficient when lags are unbalanced Key: KAFKA-10518 URL: https://issues.apache.org/jira/browse/KAFKA-10518 Project: Kafka

[jira] [Created] (KAFKA-10517) Inefficient consumer processing with fetch sessions

2020-09-23 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-10517: Summary: Inefficient consumer processing with fetch sessions Key: KAFKA-10517 URL: https://issues.apache.org/jira/browse/KAFKA-10517 Project: Kafka Issue

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-09-16 Thread Dhruvil Shah
Hi Satish, Harsha, Thanks for the KIP. Few questions below: 1. Could you describe how retention would work with this KIP and which threads are responsible for driving this work? I believe there are 3 kinds of retention processes we are looking at: (a) Regular retention for data in tiered

[jira] [Created] (KAFKA-9961) Brokers may be left in an inconsistent state after reassignment

2020-05-05 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-9961: --- Summary: Brokers may be left in an inconsistent state after reassignment Key: KAFKA-9961 URL: https://issues.apache.org/jira/browse/KAFKA-9961 Project: Kafka

[jira] [Created] (KAFKA-9956) Authorizer APIs may be invoked more than once for a given request

2020-05-05 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-9956: --- Summary: Authorizer APIs may be invoked more than once for a given request Key: KAFKA-9956 URL: https://issues.apache.org/jira/browse/KAFKA-9956 Project: Kafka

Re: [VOTE] KIP-584: Versioning scheme for features

2020-04-21 Thread Dhruvil Shah
Thanks for the KIP! +1 (non-binding) On Tue, Apr 21, 2020 at 6:09 AM David Jacot wrote: > Great KIP, thanks! +1 (non-binding) > > On Fri, Apr 17, 2020 at 8:56 PM Guozhang Wang wrote: > > > Thanks for the great KIP Kowshik, +1 (binding). > > > > On Fri, Apr 17, 2020 at 11:22 AM Jun Rao wrote:

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-04-21 Thread Dhruvil Shah
Hi Kowshik, Thanks for the KIP, this is exciting! The KIP includes examples on how operators could use the command line utility, etc. It would be great to add some high-level details on how the upgrade workflow changes overall with the addition of feature versions. - Dhruvil On Wed, Apr 15,

[jira] [Created] (KAFKA-9772) Transactional offset commit fails with IllegalStateException

2020-03-26 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-9772: --- Summary: Transactional offset commit fails with IllegalStateException Key: KAFKA-9772 URL: https://issues.apache.org/jira/browse/KAFKA-9772 Project: Kafka

Re: [DISCUSS] KIP-550: Mechanism to Delete Stray Partitions on Broker

2020-01-16 Thread Dhruvil Shah
environments, they could choose to disable the feature altogether. Let me know what you think. It would be good to hear what others think as well. Thanks, Dhruvil On Thu, Jan 16, 2020 at 3:24 AM Colin McCabe wrote: > On Wed, Jan 15, 2020, at 03:54, Dhruvil Shah wrote: > > Hi Colin, &

Re: [DISCUSS] KIP-550: Mechanism to Delete Stray Partitions on Broker

2020-01-15 Thread Dhruvil Shah
ime frame -- is it > assumed to be immediate? > > best, > Colin > > > On Tue, Jan 14, 2020, at 05:56, Dhruvil Shah wrote: > > If there are no more questions or concerns, I will start a vote thread > > tomorrow. > > > > Thanks, > > Dhruvil > > > > On

Re: [DISCUSS] KIP-550: Mechanism to Delete Stray Partitions on Broker

2020-01-14 Thread Dhruvil Shah
If there are no more questions or concerns, I will start a vote thread tomorrow. Thanks, Dhruvil On Mon, Jan 13, 2020 at 6:59 PM Dhruvil Shah wrote: > Hi Nikhil, > > Thanks for looking at the KIP. The kind of race condition you mention is > not possible as stray partition detec

Re: [DISCUSS] KIP-550: Mechanism to Delete Stray Partitions on Broker

2020-01-13 Thread Dhruvil Shah
efinitely solve this issue. > > Thanks > Nikhil > > On 2020/01/06 04:30:20, Dhruvil Shah wrote: > > Here is the link to the KIP:> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-550%3A+Mechanism+to+Delete+Stray+Partitions+on+Broker > > >

Re: [DISCUSS] KIP-550: Mechanism to Delete Stray Partitions on Broker

2020-01-05 Thread Dhruvil Shah
Here is the link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-550%3A+Mechanism+to+Delete+Stray+Partitions+on+Broker On Mon, Jan 6, 2020 at 9:59 AM Dhruvil Shah wrote: > Hi all, I would like to kick off discussion for KIP-550 which proposes a > mechanism to detect and

[DISCUSS] KIP-550: Mechanism to Delete Stray Partitions on Broker

2020-01-05 Thread Dhruvil Shah
Hi all, I would like to kick off discussion for KIP-550 which proposes a mechanism to detect and delete stray partitions on a broker. Suggestions and feedback are welcome. - Dhruvil

[jira] [Created] (KAFKA-9307) Transaction coordinator could be left in unknown state after ZK session timeout

2019-12-16 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-9307: --- Summary: Transaction coordinator could be left in unknown state after ZK session timeout Key: KAFKA-9307 URL: https://issues.apache.org/jira/browse/KAFKA-9307 Project

[jira] [Resolved] (KAFKA-8125) Check for topic existence in CreateTopicsRequest prior to creating replica assignment

2019-10-15 Thread Dhruvil Shah (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8125?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah resolved KAFKA-8125. - Resolution: Duplicate > Check for topic existence in CreateTopicsRequest prior to creat

[jira] [Created] (KAFKA-8962) KafkaAdminClient#describeTopics always goes through the controller

2019-09-30 Thread Dhruvil Shah (Jira)
Dhruvil Shah created KAFKA-8962: --- Summary: KafkaAdminClient#describeTopics always goes through the controller Key: KAFKA-8962 URL: https://issues.apache.org/jira/browse/KAFKA-8962 Project: Kafka

Re: [VOTE] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-09-12 Thread Dhruvil Shah
This is exciting! +1 (non-binding) - Dhruvil On Thu, Sep 12, 2019 at 12:29 PM Bill Bejeck wrote: > Thanks for the KIP! > > +1 (binding) > > -Bill > > On Thu, Sep 12, 2019 at 3:26 PM Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > Ismael > > > > On Mon, Sep 9, 2019, 8:28 AM

Re: [DISCUSS] KIP-487: Automatic Topic Creation on Producer

2019-07-11 Thread Dhruvil Shah
Hi Justine, Thanks for the KIP, this is great! Could you add some more information about what deprecating the broker configuration means? Would we log a warning in the logs when auto topic creation is enabled on the broker, for example? Thanks, Dhruvil On Thu, Jul 11, 2019 at 10:28 AM Justine

[jira] [Created] (KAFKA-8570) Downconversion could fail when log contains out of order message formats

2019-06-19 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-8570: --- Summary: Downconversion could fail when log contains out of order message formats Key: KAFKA-8570 URL: https://issues.apache.org/jira/browse/KAFKA-8570 Project: Kafka

[jira] [Created] (KAFKA-8359) Reconsider default for leader imbalance percentage

2019-05-13 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-8359: --- Summary: Reconsider default for leader imbalance percentage Key: KAFKA-8359 URL: https://issues.apache.org/jira/browse/KAFKA-8359 Project: Kafka Issue Type

Re: [VOTE] KIP-461 Improve Replica Fetcher behavior at handling partition failure

2019-05-08 Thread Dhruvil Shah
Thanks for the KIP! +1 (non-binding) On Wed, May 8, 2019 at 10:23 PM Colin McCabe wrote: > +1. Thanks, Aishwarya. > > Colin > > On Wed, May 8, 2019, at 17:50, Jason Gustafson wrote: > > +1. Thanks! > > > > On Wed, May 8, 2019 at 4:30 PM Aishwarya Gune > > wrote: > > > > > Hi All! > > > > > >

[jira] [Created] (KAFKA-8322) Flaky test: SslTransportLayerTest.testListenerConfigOverride

2019-05-03 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-8322: --- Summary: Flaky test: SslTransportLayerTest.testListenerConfigOverride Key: KAFKA-8322 URL: https://issues.apache.org/jira/browse/KAFKA-8322 Project: Kafka

[jira] [Resolved] (KAFKA-8185) Controller becomes stale and not able to failover the leadership for the partitions

2019-04-17 Thread Dhruvil Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah resolved KAFKA-8185. - Resolution: Not A Problem This is not a typically expected scenario and would only happen when

Re: [VOTE] KIP-434: Dead replica fetcher and log cleaner metrics

2019-03-28 Thread Dhruvil Shah
Thanks for the KIP, Viktor! This is a useful addition. +1 overall. Minor nits: > I propose to add three gauge: DeadFetcherThreadCount for the fetcher threads, log-cleaner-dead-thread-count for the log cleaner. I think you meant two instead of three. Also, would it make sense to name these

Re: problems in Kafka unit testing trunk

2018-11-27 Thread Dhruvil Shah
The unit test itself does not seem to use too many files. What is the output for `ulimit -n` on your system? Running `lsof` might also be helpful to determine how many open files you have while Kafka is not running. - Dhruvil On Tue, Nov 27, 2018 at 9:20 AM lk gen wrote: > When running

Re: Help on 'Error while writing to checkpoint file' Issue

2018-10-30 Thread Dhruvil Shah
Hi Dasun, seems like the screenshots were not attached. Could you please open a Jira here: https://issues.apache.org/jira/projects/KAFKA Thanks, Dhruvil On Tue, Oct 30, 2018 at 10:29 PM Dasun Nirmitha wrote: > Hello Guys > I'm currently testing a Java Kafka producer application coded to

Re: [ANNOUNCE] New committer: Colin McCabe

2018-09-25 Thread Dhruvil Shah
Congratulations Colin! On Tue, Sep 25, 2018 at 6:47 AM Stephane Maarek < steph...@simplemachines.com.au> wrote: > Congrats Colin ! > > On Tue., 25 Sep. 2018, 3:33 pm Bill Bejeck, wrote: > > > Congrats Colin! > > > > On Tue, Sep 25, 2018 at 8:11 AM Manikumar > > wrote: > > > > > Congrats Colin!

Re: [VOTE] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-09-17 Thread Dhruvil Shah
he KIP! > On Sun, Sep 16, 2018 at 7:40 PM Matthias J. Sax > wrote: > > > > +1 (binding) > > > > -Matthias > > > > On 9/14/18 4:57 PM, Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > > > Ismael > > &

[VOTE] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-09-14 Thread Dhruvil Shah
Hi all, I would like to start a vote on KIP-361. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-361%3A+Add+Consumer+Configuration+to+Disable+Auto+Topic+Creation Thanks, Dhruvil

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-09-13 Thread Dhruvil Shah
lly, I think positive configs (`allow` instead of `suppress`) > >>>> are > >>>>>>> easier to understand. > >>>>>>> > >>>>>>> Ismael > >>>>>>> > >>>>>>> O

[jira] [Created] (KAFKA-7385) Log compactor crashes when empty headers are retained with idempotent / transaction producers

2018-09-07 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-7385: --- Summary: Log compactor crashes when empty headers are retained with idempotent / transaction producers Key: KAFKA-7385 URL: https://issues.apache.org/jira/browse/KAFKA-7385

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
te.topics=true above to disallow auto topic creation. On Wed, Aug 22, 2018 at 10:34 PM Dhruvil Shah wrote: > To be clear, we will allow auto topic creation only when server config > auto.create.topics.enable=true and consumer config > allow.auto.create.topics=true; when either

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
> > > > or > > > > "allow.auto.topic.creation" > > > > w.r.t. suppress or allow, I don't have strong opinion either. It's just a > > matter of choosing the proper default value. > > > > Cheers > > > >> On Wed, Aug 22, 2

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
first step (or warn > every time it's used). > > Ismael > > On 21 Aug 2018 3:56 pm, "Dhruvil Shah" wrote: > > Hi, > > I would like to start discussion on KIP-361 that proposes we add a consumer > configuration to disable auto topic creation. > > Link

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-22 Thread Dhruvil Shah
quot;enable. The consumer > > cannot enable auto topic creation because it is configured on the broker. > > All it can do is prevent it from happening if it is enabled. > > > > -Jason > > > > On Tue, Aug 21, 2018 at 3:56 PM, Dhruvil Shah > wrote: > >

Re: [ANNOUNCE] New Kafka PMC member: Dong Lin

2018-08-21 Thread Dhruvil Shah
Congratulations, Dong! On Tue, Aug 21, 2018 at 4:38 PM Jason Gustafson wrote: > Congrats! > > On Tue, Aug 21, 2018 at 10:03 AM, Ray Chiang wrote: > > > Congrats Dong! > > > > -Ray > > > > > > On 8/21/18 9:33 AM, Becket Qin wrote: > > > >> Congrats, Dong! > >> > >> On Aug 21, 2018, at 11:03 PM,

[DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-08-21 Thread Dhruvil Shah
Hi, I would like to start discussion on KIP-361 that proposes we add a consumer configuration to disable auto topic creation. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-361%3A+Add+Consumer+Configuration+to+Disable+Auto+Topic+Creation Suggestions and feedback are

[jira] [Created] (KAFKA-7320) Provide ability to disable auto topic creation in KafkaConsumer

2018-08-21 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-7320: --- Summary: Provide ability to disable auto topic creation in KafkaConsumer Key: KAFKA-7320 URL: https://issues.apache.org/jira/browse/KAFKA-7320 Project: Kafka

Re: [VOTE] KIP-346 - Improve LogCleaner behavior on error

2018-08-13 Thread Dhruvil Shah
Thanks for the KIP, Stanislav! +1 (non-binding) - Dhruvil On Mon, Aug 13, 2018 at 9:39 AM Colin McCabe wrote: > +1 (non-binding) > > best, > Colin > > On Tue, Aug 7, 2018, at 04:19, Stanislav Kozlovski wrote: > > Hey everybody, > > I'm starting a vote on KIP-346 > > < >

Re: [DISCUSS] KIP-346 - Limit blast radius of log compaction failure

2018-07-25 Thread Dhruvil Shah
For the cleaner thread specifically, I do not think respawning will help at all because we are more than likely to run into the same issue again which would end up crashing the cleaner. Retrying makes sense for transient errors or when you believe some part of the system could have healed itself,

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-20 Thread Dhruvil Shah
If we are looking into deprecating the empty group id, would it also make sense to have the same character restriction for it as that for topic names? We have stricter validation for topic names but none for group id and transaction id. I think we should (eventually) make character restriction the

[jira] [Created] (KAFKA-7185) getMatchingAcls throws StringIndexOutOfBoundsException for empty resource name

2018-07-19 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-7185: --- Summary: getMatchingAcls throws StringIndexOutOfBoundsException for empty resource name Key: KAFKA-7185 URL: https://issues.apache.org/jira/browse/KAFKA-7185 Project

[jira] [Resolved] (KAFKA-7184) Kafka is going down with issue ERROR Failed to clean up log for __consumer_offsets-0 in dir /tmp/kafkadev2-logs due to IOException (kafka.server.LogDirFailureChannel)

2018-07-19 Thread Dhruvil Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7184?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah resolved KAFKA-7184. - Resolution: Not A Problem > Kafka is going down with issue ERROR Failed to clean up

Re: [DISCUSS] KIP-263: Allow broker to skip sanity check of inactive segments on broker startup

2018-06-27 Thread Dhruvil Shah
+1 to what Jason said. We need a better long-term strategy for dealing with corrupted log and index data, but the sanity checks we have do not guarantee much in this regard. For now, we could do away with these index sanity checks in my opinion. We could handle the missing index case at startup.

[jira] [Created] (KAFKA-7076) Broker startup could be inefficient when using old message format

2018-06-19 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-7076: --- Summary: Broker startup could be inefficient when using old message format Key: KAFKA-7076 URL: https://issues.apache.org/jira/browse/KAFKA-7076 Project: Kafka

[jira] [Resolved] (KAFKA-6881) Kafka 1.1 Broker version crashes when deleting log

2018-06-19 Thread Dhruvil Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah resolved KAFKA-6881. - Resolution: Not A Bug Closing this JIRA because /tmp was being used as the log directory

[jira] [Created] (KAFKA-7045) Consumer may not be able to consume all messages when down-conversion is required

2018-06-11 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-7045: --- Summary: Consumer may not be able to consume all messages when down-conversion is required Key: KAFKA-7045 URL: https://issues.apache.org/jira/browse/KAFKA-7045

[jira] [Created] (KAFKA-7030) Add configuration to disable message down-conversion

2018-06-09 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-7030: --- Summary: Add configuration to disable message down-conversion Key: KAFKA-7030 URL: https://issues.apache.org/jira/browse/KAFKA-7030 Project: Kafka Issue Type

Re: [VOTE] KIP-266: Add TimeoutException for KafkaConsumer#position

2018-06-05 Thread Dhruvil Shah
I agree that using `default.timeout.ms` could cause confusion since we already have other timeout configurations in the consumer. +1 for using `default.block.ms`. Thanks, Dhruvil On Tue, Jun 5, 2018 at 11:48 AM, Bill Bejeck wrote: > Hi Jason, > > At first, I thought the same name between the

[jira] [Created] (KAFKA-6950) Add mechanism to delay response to failed client authentication

2018-05-25 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-6950: --- Summary: Add mechanism to delay response to failed client authentication Key: KAFKA-6950 URL: https://issues.apache.org/jira/browse/KAFKA-6950 Project: Kafka

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-22 Thread Dhruvil Shah
t; > > -Jason > > > > On Mon, May 21, 2018 at 3:59 PM, Ismael Juma <ism...@juma.me.uk> wrote: > > > > > Thanks for the KIP, +1 (binding). > > > > > > Ismael > > > > > > On Mon, May 21, 2018 at 7:52 AM Dhruvil Shah <dhru...@

[jira] [Created] (KAFKA-6927) Broker uses significant amount of memory during down-conversion

2018-05-21 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-6927: --- Summary: Broker uses significant amount of memory during down-conversion Key: KAFKA-6927 URL: https://issues.apache.org/jira/browse/KAFKA-6927 Project: Kafka

[VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Dhruvil Shah
Hi, I would like to start a vote on KIP-306 which proposes to add a configuration to delay responses to failed authentication. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-306%3A+Configuration+for+Delaying+Response+to+Failed+Client+Authentication Thanks, Dhruvil

[DISCUSS] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-19 Thread Dhruvil Shah
Hi, I created a KIP that proposes we add a broker configuration to delay responses to failed client authentication. This will help prevent DoS-like situations because of a misconfigured application trying to connect with incorrect or stale credentials over and over again. Link to the KIP:

Re: [VOTE] KIP-283: Efficient Memory Usage for Down-Conversion

2018-05-16 Thread Dhruvil Shah
> > +1 (binding) > > Ismael > > On Wed, May 2, 2018 at 9:27 AM Dhruvil Shah <dhru...@confluent.io> wrote: > > > Hi all, > > > > I would like to start the vote on KIP-238: Efficient Memory Usage for > > Down-Conversion. > > > > For reference,

Re: [VOTE] KIP-283: Efficient Memory Usage for Down-Conversion

2018-05-09 Thread Dhruvil Shah
s use > > "enable" rather than "enabled." > > > > -Jason > > > > On Wed, May 2, 2018 at 9:35 AM, Ted Yu <yuzhih...@gmail.com> wrote: > > > > > +1 > > > > > > On Wed, May 2, 2018 at 9:27 AM, Dhruvil Shah <

[VOTE] KIP-283: Efficient Memory Usage for Down-Conversion

2018-05-02 Thread Dhruvil Shah
Hi all, I would like to start the vote on KIP-238: Efficient Memory Usage for Down-Conversion. For reference, the link to the KIP is here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-283%3A+Efficient+Memory+Usage+for+Down-Conversion and the discussion thread is here:

Re: [DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-30 Thread Dhruvil Shah
-converting the entire partition upfront. Jun, let me know what you think. Any other suggestions / feedback are welcome. Thanks, Dhruvil On Tue, Apr 17, 2018 at 4:21 PM, Dhruvil Shah <dhru...@confluent.io> wrote: > Hi Jun, > > Yes, that is true. Ideally, we should be able to do

Re: [VOTE] KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User

2018-04-20 Thread Dhruvil Shah
+1 from me as well. I was going to file a JIRA for this exact same problem. :-) On Thu, Apr 19, 2018 at 10:45 PM, Alex Dunayevsky wrote: > +1 > > > 4 votes total: > > 1 binding vote (Jason Gustafson) > > 3 non-binding votes (Moshe Blumberg, Ted Yu, Alex Dunayevsky) >

Re: [DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-17 Thread Dhruvil Shah
? > > Jun > > > On Fri, Apr 6, 2018 at 2:56 PM, Dhruvil Shah <dhru...@confluent.io> wrote: > > > Hi, > > > > I created a KIP to help mitigate out of memory issues during > > down-conversion. The KIP proposes introducing a configuration that can >

Re: [DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-11 Thread Dhruvil Shah
alks > > about pre-allocated fixed size buffers, but your last note suggests that > > you would use temporary buffers created for each partition. Do we need to > > consider using a memory pool for these or do we think that the buffers > will > > be small enough to cope wi

Re: [DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-11 Thread Dhruvil Shah
th downconversions? This > will be a clear improvement over what we have now in any case, but just > checking anyway. > > Regards, > > Rajini > > On Sat, Apr 7, 2018 at 12:29 AM, Dhruvil Shah <dhru...@confluent.io> > wrote: > > > Hi Ted, > >

Re: [DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-06 Thread Dhruvil Shah
t; ? > > bq. even if it exceeds fetch.max.bytes > > I did a brief search but didn't see the above config. Did you mean > message.max.bytes > ? > > bq. with possibility to grow if the allocation > > After the buffers grow, is there a way to trim them down if subsequent

Re: [DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-06 Thread Dhruvil Shah
I fixed the diagrams - let me know if you are still having trouble seeing them. Thanks, Dhruvil On Fri, Apr 6, 2018 at 3:05 PM, Ted Yu <yuzhih...@gmail.com> wrote: > The two embedded diagrams seem broken. > > Can you double check ? > > Thanks > > On Fri, Apr 6, 20

[DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-06 Thread Dhruvil Shah
Hi, I created a KIP to help mitigate out of memory issues during down-conversion. The KIP proposes introducing a configuration that can prevent down-conversions altogether, and also describes a design for efficient memory usage for down-conversion.