Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-10 Thread Apurva Mehta
Hi Dong, Thanks for your comments. Yes, with retries=MAX_INT, producer.flush() may block. I think there are two solutions: a good one would be to adopt some form of KIP-91 to bound the time a message can remain unacknowledged. Alternately, we could set the default retries to 10 or something. I

[jira] [Created] (KAFKA-5724) AbstractPartitionAssignor does not take into consideration that partition number may start from non-zero

2017-08-10 Thread Allen Wang (JIRA)
Allen Wang created KAFKA-5724: - Summary: AbstractPartitionAssignor does not take into consideration that partition number may start from non-zero Key: KAFKA-5724 URL: https://issues.apache.org/jira/browse/KAFKA-5724

Re: [DISCUSS] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-08-10 Thread Jun Rao
Hi, Tom, Thanks for the KIP. Looks good overall. A few minor comments. 1. In most requests with topic partitions, we nest partitions inside topic. So, instead of [topic partition_id], we do [topic, [partition_id]] to save some space. 2. The preferred leader election just tries to move the

Re: [DISCUSS] KIP-186: Increase offsets retention default to 7 days

2017-08-10 Thread Guozhang Wang
+1 from me On Wed, Aug 9, 2017 at 9:40 AM, Jason Gustafson wrote: > +1 on the bump to 7 days. Wanted to mention one minor point. The > OffsetCommit RPC still provides the ability to set the retention time from > the client, but we do not use it in the consumer. Should we

Re: [VOTE] KIP-163: Lower the Minimum Required ACL Permission of OffsetFetch

2017-08-10 Thread Jun Rao
Hi, Vahid, Thanks for the KIP. +1 from me. Jun On Wed, Aug 2, 2017 at 2:40 PM, Vahid S Hashemian wrote: > Hi all, > > Thanks to everyone who participated in the discussion on KIP-163, and > provided feedback. > The KIP can be found at >

Re: [DISCUSS] KIP-186: Increase offsets retention default to 7 days

2017-08-10 Thread James Cheng
+1 from me! -James > On Aug 8, 2017, at 5:24 PM, Ewen Cheslack-Postava wrote: > > Hi all, > > I posted a simple new KIP for a problem we see with a lot of users: > KIP-186: Increase offsets retention default to 7 days > >

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-10 Thread Jun Rao
Hi, Sumant, Thanks for the KIP. Nice documentation on all current issues with the timeout. You also brought up a good use case for timing out a message. For applications that collect and send sensor data to Kafka, if the data can't be sent to Kafka for some reason, the application may prefer to

Re: [DISCUSS] KIP-186: Increase offsets retention default to 7 days

2017-08-10 Thread Vahid S Hashemian
+1 on both the KIP and Jason's suggestion of adding the consumer config. Thanks. --Vahid From: Jason Gustafson To: dev@kafka.apache.org Date: 08/09/2017 09:40 AM Subject:Re: [DISCUSS] KIP-186: Increase offsets retention default to 7 days +1 on the bump

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-10 Thread Dong Lin
Hey Jun, This is a very good idea. I have updated the KIP-113 so that DescribeDirResponse returns lag instead of LEO. If the replica is not a temporary replica, then lag = max(0, HW - LEO). Otherwise, lag = primary Replica's LEO - temporary Replica's LEO. Thanks! Dong On Thu, Aug 10, 2017 at

[GitHub] kafka pull request #3644: KAFKA-5711: batch restore should handle deletes

2017-08-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3644 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[GitHub] kafka pull request #3646: MINOR: Remove unneeded error handlers in deprecate...

2017-08-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3646 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-10 Thread Jun Rao
Hi, Tom, Dong, A couple of comments on that. 1. I think we can unify the reporting of lags. Basically, the lag will be reported on every replica (temporary or permanent), not just at the leader replica. If it's permanent, lag is max(0, HW - LEO) as it is now. Otherwise, lag is (LEO of permanent

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-10 Thread Tom Bentley
I've spent some time thinking about KIP-179 and KIP-113, the proposed algorithms and APIs, and trying to weigh the pros and cons of various alternative options. I think Dong's reasons for the algorithm for inter-broker move in KIP-113 make a lot of sense. I don't think it would be at all simple

Re: [DISCUSS] KIP-182: Reduce Streams DSL overloads and allow easier use of custom storage engines

2017-08-10 Thread Damian Guy
> Got it, thanks. > > Does it still make sense to have one static constructors for each spec, > with one constructor having only one parameter to make it more usable, i.e. > as a user I do not need to give all parameters if I only want to override > one of them? Maybe we can just name the

[GitHub] kafka pull request #3654: KAFKA-5562: Do streams state directory cleanup on ...

2017-08-10 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3654 KAFKA-5562: Do streams state directory cleanup on a single thread Backported from trunk: https://github.com/apache/kafka/pull/3516 You can merge this pull request into a Git repository by running:

[GitHub] kafka pull request #3653: KAFKA-5152: move state restoration out of rebalanc...

2017-08-10 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3653 KAFKA-5152: move state restoration out of rebalance and into poll loop In `onPartitionsAssigned`: 1. release all locks for non-assigned suspended tasks. 2. resume any suspended tasks. 3.

[jira] [Resolved] (KAFKA-3389) ReplicaStateMachine areAllReplicasForTopicDeleted check not handling well case when there are no replicas for topic

2017-08-10 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3389?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3389. -- Resolution: Won't Fix As mentioned in the previous comment, this may not be an issue. Pl reopen if

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-10 Thread Tom Bentley
Hi Dong and Jun, It seems that KIP-179 does not explicitly specify the definition of this > lag. Given that the definition of "caught up" is "is the replica in the ISR?", I found the code in Partition.maybeExpandIsr() which decides whether a replica should be added to the to the ISR and it uses

[GitHub] kafka pull request #3652: MINOR: change log level in ThreadCache to trace

2017-08-10 Thread dguy
Github user dguy closed the pull request at: https://github.com/apache/kafka/pull/3652 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Created] (KAFKA-5723) Refactor BrokerApiVersionsCommand to use the KafkaAdminClient

2017-08-10 Thread Viktor Somogyi (JIRA)
Viktor Somogyi created KAFKA-5723: - Summary: Refactor BrokerApiVersionsCommand to use the KafkaAdminClient Key: KAFKA-5723 URL: https://issues.apache.org/jira/browse/KAFKA-5723 Project: Kafka

[jira] [Created] (KAFKA-5722) Refactor ConfigCommand to use the new AdminClient

2017-08-10 Thread Viktor Somogyi (JIRA)
Viktor Somogyi created KAFKA-5722: - Summary: Refactor ConfigCommand to use the new AdminClient Key: KAFKA-5722 URL: https://issues.apache.org/jira/browse/KAFKA-5722 Project: Kafka Issue

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-10 Thread Dong Lin
Hey Apurva, Thanks for the KIP. I have read through the KIP and the prior discussion in this thread. I have three concerns that are related to Becket's comments: - Is it true that, as Becket has mentioned, producer.flush() may block infinitely if retries=MAX_INT? This seems like a possible

[GitHub] kafka pull request #3652: MINOR: change log level in ThreadCache to trace

2017-08-10 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3652 MINOR: change log level in ThreadCache to trace cache eviction logging at debug level is too high volume You can merge this pull request into a Git repository by running: $ git pull

Re: Usage of SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG

2017-08-10 Thread M. Manna
Thanks got it :) On 10 Aug 2017 7:02 am, "Manikumar" wrote: > check here: > https://github.com/apache/kafka/blob/trunk/clients/src/ > main/java/org/apache/kafka/common/security/ssl/SslFactory.java#L84 > > On Thu, Aug 10, 2017 at 3:49 AM, M. Manna

Re: Usage of SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG

2017-08-10 Thread Manikumar
check here: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java#L84 On Thu, Aug 10, 2017 at 3:49 AM, M. Manna wrote: > Hello, > > I have been trying to find the usage of this property within Kafka source. >