Re: [VOTE] 0.10.0.0 RC2

2016-05-02 Thread Ismael Juma
+1 On 2 May 2016 07:18, "Gwen Shapira" wrote: > Thanks for the catches :) > > Is it ok if we delay rolling out the new RC to the end of the week, to > allow more time for testing? > > Gwen > > On Sun, May 1, 2016 at 11:49 AM, Ewen Cheslack-Postava > wrote: > > We had a blocking issue on the rele

Re: [VOTE] 0.10.0.0 RC2

2016-05-02 Thread Ismael Juma
On second thought, will people be able to test the binary packages if they can't start the broker? Ismael On Mon, May 2, 2016 at 5:43 AM, Ismael Juma wrote: > +1 > On 2 May 2016 07:18, "Gwen Shapira" wrote: > >> Thanks for the catches :) >> >> Is it ok if we delay rolling out the new RC to the

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Bill Warshaw
Conditions 1, 2 and 3 will all be checked sequentially. If any of the three conditions is true, that segment will be deleted. This is what it looks like in my commit: https://github.com/apache/kafka/blob/a229462df567f91f76122668037e1bcbbbdff41b/core/src/main/scala/kafka/log/LogManager.scala#L423-

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Guozhang Wang
Can we do 1 and 2 in one pass, and 3 in another pass? It may result in different results but semantically it should be acceptable. Arguably saving one pass on the segment list may not be huge, but if it is straight-forward to do I'd suggest choose this option. Guozhang On Mon, May 2, 2016 at 7:

Re: [VOTE] 0.10.0.0 RC2

2016-05-02 Thread Dana Powers
I was unable to use/test the rc2 binary artifact without manually patching kafka-run-class.sh . I'd vote for a new release candidate. -Dana On Mon, May 2, 2016 at 5:44 AM, Ismael Juma wrote: > On second thought, will people be able to test the binary packages if they > can't start the broker? >

Re: [VOTE] 0.10.0.0 RC2

2016-05-02 Thread Gwen Shapira
Makes sense. I'll try to get one out later today. On Mon, May 2, 2016 at 8:16 AM, Dana Powers wrote: > I was unable to use/test the rc2 binary artifact without manually > patching kafka-run-class.sh . I'd vote for a new release candidate. > > -Dana > > On Mon, May 2, 2016 at 5:44 AM, Ismael Jum

Re: [VOTE] KIP-50: Move Authorizer to o.a.k.common package - Round 2

2016-05-02 Thread Ashish Singh
KIP-50 has passed with +6 (binding) and +3 (non-binding). Thanks everyone for reviews and votes! On Sun, May 1, 2016 at 4:08 PM, Don Bosco Durai wrote: > +1 (non binding) > > > > > On 4/28/16, 7:47 AM, "Jun Rao" wrote: > > >Ashish, > > > >Thanks for the proposal. +1 > > > >Jun > > > >On Mon, A

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Bill Warshaw
Yeah 1 and 2 could easily be combined into the same predicate. On Mon, May 2, 2016 at 10:27 AM, Guozhang Wang wrote: > Can we do 1 and 2 in one pass, and 3 in another pass? It may result in > different results but semantically it should be acceptable. Arguably saving > one pass on the segment li

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Guozhang Wang
Thanks. I'm +1 on this proposal given the comment above. On Mon, May 2, 2016 at 9:34 AM, Bill Warshaw wrote: > Yeah 1 and 2 could easily be combined into the same predicate. > > On Mon, May 2, 2016 at 10:27 AM, Guozhang Wang wrote: > > > Can we do 1 and 2 in one pass, and 3 in another pass? It

Re: [VOTE] 0.10.0.0 RC2

2016-05-02 Thread Harsha
I would also like to get a decision on KIP-45. From the discussions it seems we are ok to include the changes proposed. -Harsha On Mon, May 2, 2016, at 09:15 AM, Gwen Shapira wrote: > Makes sense. > I'll try to get one out later today. > > > > On Mon, May 2, 2016 at 8:16 AM, Dana Powers > wro

Re: [DISCUSS] KIP-53 Add custom policies for reconnect attempts to NetworkdClient

2016-05-02 Thread Guozhang Wang
For the specific problem of connection storm, randomized with normal distribution at specified mean as "reconnect.backoff.ms" has been proved pretty well. The most recent usage of it in my mind is RAFT, and it turns out pretty effective in eliminating leader-election storms. Guozhang On Fri, Apr

Re: [DISCUSS] KIP-54 Sticky Partition Assignment Strategy

2016-05-02 Thread Guozhang Wang
Just FYI, the StreamsPartitionAssignor in Kafka Streams are already doing some sort of sticky partitioning mechanism. This is done through the userData field though; i.e. all group members send their current "assigned partitions" in their join group request, which will be grouped and send to the le

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-05-02 Thread Guozhang Wang
Just my two cents here: I agree with Ewen and Grant on the indication of the "unstable" annotations of being possible for backward incompatible. That means, users can make a call themselves of whether to start trying out the new APIs / libraries with the risk or changing code when it changes in a

[jira] [Commented] (KAFKA-3646) Console producer using new producer should set timestamp

2016-05-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267074#comment-15267074 ] Guozhang Wang commented on KAFKA-3646: -- [~miguno][~gwenshap] When creating this ticke

[jira] [Created] (KAFKA-3647) Unable to set a ssl provider

2016-05-02 Thread Elvar (JIRA)
Elvar created KAFKA-3647: Summary: Unable to set a ssl provider Key: KAFKA-3647 URL: https://issues.apache.org/jira/browse/KAFKA-3647 Project: Kafka Issue Type: Bug Components: security

[jira] [Resolved] (KAFKA-3640) Reduce the latency of topic metadata requests

2016-05-02 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3640?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-3640. - Resolution: Fixed Issue resolved by pull request 1303 [https://github.com/apache/kafka/pull/1303]

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Jay Kreps
Two comments: 1. Is there a reason to use physical time rather than offset? The idea is for the consumer to say when it has consumed something so it can be deleted, right? It seems like offset would be a much more precise way to do this--i.e. the consumer says "I have checkpointed stat

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Bill Warshaw
1. Initially I looked at using the actual offset, by adding a call to AdminUtils to just delete anything in a given topic/partition to a given offset. I ran into a lot of trouble here trying to work out how the system would recognize that every broker had successfully deleted that range from the

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Jay Kreps
I think you are saying you considered a kind of trim() api that would synchronously chop off the tail of the log starting from a given offset. That would be one option, but what I was saying was slightly different: in the proposal you have where there is a config that controls retention that the us

Re: [DISCUSS] KIP-54 Sticky Partition Assignment Strategy

2016-05-02 Thread Vahid S Hashemian
Hi Ewen, Thank you for reviewing the KIP and providing feedback. I believe the need to commit would still be there, as you mentioned. The main advantage, however, would be when dealing with local state based on partitions assigned, as described in https://cwiki.apache.org/confluence/display/KA

[jira] [Updated] (KAFKA-3358) Only request metadata updates once we have topics or a pattern subscription

2016-05-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3358?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3358: --- Fix Version/s: (was: 0.10.0.0) (was: 0.10.1.0) > Only request metadata upda

Re: [DISCUSS] KIP-54 Sticky Partition Assignment Strategy

2016-05-02 Thread Vahid S Hashemian
Hi Guozhang, Thanks for the pointer. I'll try to take a closer look and get a better understanding and see if there is anything that can be leveraged for KIP-54 implementation. Regards, Vahid Hashemian From: Guozhang Wang To: "dev@kafka.apache.org" Date: 05/02/2016 10:34 AM Subjec

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Bill Warshaw
The problem with offset as a config option is that offsets are partition-specific, so we'd need a per-partition config. This would work for our particular use case, where we have single-partition topics, but for multiple-partition topics it would delete from all partitions based on a global topic-

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Jay Kreps
Gotcha, good point. But barring that limitation, you agree that that makes more sense? -Jay On Mon, May 2, 2016 at 2:29 PM, Bill Warshaw wrote: > The problem with offset as a config option is that offsets are > partition-specific, so we'd need a per-partition config. This would work > for our

Re: [VOTE] KIP-47 - Add timestamp-based log deletion policy

2016-05-02 Thread Bill Warshaw
Yes, I'd agree that offset is a more precise configuration than timestamp. If there was a way to set a partition-level configuration, I would rather use log.retention.min.offset than timestamp. If you have an approach in mind I'd be open to investigating it. On Mon, May 2, 2016 at 5:33 PM, Jay Kr

[jira] [Commented] (KAFKA-3185) Allow users to cleanup internal data

2016-05-02 Thread Henry Cai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267619#comment-15267619 ] Henry Cai commented on KAFKA-3185: -- Second to this. It's very painful for us to test our

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-05-02 Thread Magnus Edenhill
Rajini, I think I found a small documentation error on the KIP-43 wiki page, it says the SASL framing size is int16, but I believe it should be int32. Can you verify? Regards, Magnus 2016-04-25 15:38 GMT+02:00 Rajini Sivaram : > Magnus, > > I have updated KIP-43 to include a section with the

[jira] [Created] (KAFKA-3648) maxTimeToBlock should be enforced

2016-05-02 Thread chen zhu (JIRA)
chen zhu created KAFKA-3648: --- Summary: maxTimeToBlock should be enforced Key: KAFKA-3648 URL: https://issues.apache.org/jira/browse/KAFKA-3648 Project: Kafka Issue Type: Bug Reporter:

[jira] [Updated] (KAFKA-3648) maxTimeToBlock should be enforced

2016-05-02 Thread chen zhu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3648?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] chen zhu updated KAFKA-3648: Description: Currently the maxTimeToBlock in allocate(int size, long maxTimeToBlock) in clients/producer/int

[jira] [Work started] (KAFKA-3648) maxTimeToBlock should be enforced

2016-05-02 Thread chen zhu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3648?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3648 started by chen zhu. --- > maxTimeToBlock should be enforced > -- > >

[jira] [Commented] (KAFKA-3647) Unable to set a ssl provider

2016-05-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267749#comment-15267749 ] Ismael Juma commented on KAFKA-3647: Before you change the provider, have you tried in

[GitHub] kafka pull request: KAFKA-3648: maxTimeToBlock should be enforced

2016-05-02 Thread zhuchen1018
GitHub user zhuchen1018 opened a pull request: https://github.com/apache/kafka/pull/1304 KAFKA-3648: maxTimeToBlock should be enforced maxTimeToBlock in allocate(int size, long maxTimeToBlock) is enforced. You can merge this pull request into a Git repository by running: $ git

[jira] [Commented] (KAFKA-3648) maxTimeToBlock should be enforced

2016-05-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267783#comment-15267783 ] ASF GitHub Bot commented on KAFKA-3648: --- GitHub user zhuchen1018 opened a pull reque

[jira] [Comment Edited] (KAFKA-3565) Producer's throughput lower with compressed data after KIP-31/32

2016-05-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267803#comment-15267803 ] Jun Rao edited comment on KAFKA-3565 at 5/3/16 12:25 AM: - [~becket

[jira] [Commented] (KAFKA-3565) Producer's throughput lower with compressed data after KIP-31/32

2016-05-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267803#comment-15267803 ] Jun Rao commented on KAFKA-3565: Jiangjie, thanks for the results. It seems that the overa

[GitHub] kafka pull request: KAFKA-3448: add % character to ipv6 regex

2016-05-02 Thread soumyajit-sahu
Github user soumyajit-sahu closed the pull request at: https://github.com/apache/kafka/pull/1120 --- 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

[jira] [Commented] (KAFKA-3448) IPV6 Regex is missing % character

2016-05-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267855#comment-15267855 ] ASF GitHub Bot commented on KAFKA-3448: --- Github user soumyajit-sahu closed the pull

[GitHub] kafka pull request: KAFKA-3448: Fix ipv6 regex pattern trunk

2016-05-02 Thread soumyajit-sahu
GitHub user soumyajit-sahu opened a pull request: https://github.com/apache/kafka/pull/1305 KAFKA-3448: Fix ipv6 regex pattern trunk IPV6 address can contain % character. This PR is in lieu of the now abandoned PR 1120 You can merge this pull request into a Git repository by run

[jira] [Commented] (KAFKA-3448) IPV6 Regex is missing % character

2016-05-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15267884#comment-15267884 ] ASF GitHub Bot commented on KAFKA-3448: --- GitHub user soumyajit-sahu opened a pull re

[jira] [Commented] (KAFKA-3585) Shutdown slow when there is only one broker which is controller

2016-05-02 Thread Taiyuan Zhang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268068#comment-15268068 ] Taiyuan Zhang commented on KAFKA-3585: -- I tried to reproduce the issue, but the shutt

[GitHub] kafka pull request: MINOR: Add version check on enable-systest-eve...

2016-05-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1298 --- 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 enab

Build failed in Jenkins: kafka-trunk-jdk8 #576

2016-05-02 Thread Apache Jenkins Server
See Changes: [me] MINOR: Add version check on enable-systest-events flag -- [...truncated 4276 lines...] kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned PASSED kafka.a

[jira] [Commented] (KAFKA-3585) Shutdown slow when there is only one broker which is controller

2016-05-02 Thread Pengwei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268171#comment-15268171 ] Pengwei commented on KAFKA-3585: Is the last broker become the controller ? I can repro

Jenkins build is back to normal : kafka-0.10.0-jdk7 #39

2016-05-02 Thread Apache Jenkins Server
See

[GitHub] kafka pull request: MINOR: Clean up of Kafka Consumer

2016-05-02 Thread Ishiihara
GitHub user Ishiihara opened a pull request: https://github.com/apache/kafka/pull/1306 MINOR: Clean up of Kafka Consumer You can merge this pull request into a Git repository by running: $ git pull https://github.com/Ishiihara/kafka minor-consumer-cleanup Alternatively you ca