Jenkins build is back to normal : kafka-trunk-jdk7 #1239

2016-05-03 Thread Apache Jenkins Server
See

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

2016-05-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268242#comment-15268242 ] Jiangjie Qin commented on KAFKA-3565: - [~junrao] I updated the results in the Google s

[GitHub] kafka pull request: KAFKA-3640: Reduce the latency of topic metada...

2016-05-03 Thread fpj
Github user fpj closed the pull request at: https://github.com/apache/kafka/pull/1303 --- 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 enabled

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

2016-05-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3640?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268258#comment-15268258 ] ASF GitHub Bot commented on KAFKA-3640: --- Github user fpj closed the pull request at:

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

2016-05-03 Thread Rajini Sivaram
Magnus, Yes, you are absolutely right. I have fixed the wiki page. Thank you for pointing it out. Regards, Rajini On Mon, May 2, 2016 at 11:41 PM, Magnus Edenhill wrote: > Rajini, > > I think I found a small documentation error on the KIP-43 wiki page, it > says the SASL framing size is int16

[jira] [Updated] (KAFKA-3522) Consider adding version information into rocksDB storage format

2016-05-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3522?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-3522: Assignee: (was: Eno Thereska) > Consider adding version information into rocksDB storage format

Re: [DISCUSS] KIP-55: Secure quotas for authenticated users

2016-05-03 Thread Rajini Sivaram
Ewen, Thank you for the review. I agree that ideally we would have one definition of quotas that handles all cases. But I couldn't quite fit all the combinations that are possible today with client-id-based quotas into the new configuration. I think upgrade path is not bad since quotas are per-bro

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

2016-05-03 Thread Johan Abbors (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268594#comment-15268594 ] Johan Abbors commented on KAFKA-3647: - I'm also having problems using Kafka with SSL f

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

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268712#comment-15268712 ] Ismael Juma commented on KAFKA-3647: [~johabb], I don't think it's a JVM issue as libr

Re: [DISCUSS] mbeans overwritten with identical clients on a single jvm

2016-05-03 Thread Rajini Sivaram
The proposal in KIP-55 is to have hierarchical quotas with two levels of hierarchy - quotas assigned to users with sub-quotas assigned to clients (client-ids) of users. Typically, clients within a JVM run as a single user, but within a JVM, clients could use different client-ids for the purpose of

[jira] [Assigned] (KAFKA-3262) Make KafkaStreams debugging friendly

2016-05-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3262?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-3262: --- Assignee: Eno Thereska > Make KafkaStreams debugging friendly > -

[jira] [Created] (KAFKA-3649) Add capability to query broker process for configuration properties

2016-05-03 Thread David Tucker (JIRA)
David Tucker created KAFKA-3649: --- Summary: Add capability to query broker process for configuration properties Key: KAFKA-3649 URL: https://issues.apache.org/jira/browse/KAFKA-3649 Project: Kafka

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-05-03 Thread Michael Graff (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268959#comment-15268959 ] Michael Graff commented on KAFKA-3209: -- We have a use case which is in effect "take a

Re: [DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-05-03 Thread parth brahmbhatt
Bumping this up one more time, can other committers review? Thanks Parth On Tue, Apr 26, 2016 at 9:07 AM, Harsha wrote: > Parth, > Overall current design looks good to me. I am +1 on the KIP. > > Gwen , Jun can you review this as well. > > -Harsha > > On Tue, Apr 19, 2016, at 09:57 AM

Re: [DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-05-03 Thread Giampaolo Trapasso
Hi, I'm really interested in this KIP too. Giampaolo 2016-05-03 18:12 GMT+02:00 parth brahmbhatt : > Bumping this up one more time, can other committers review? > > Thanks > Parth > > On Tue, Apr 26, 2016 at 9:07 AM, Harsha wrote: > > > Parth, > > Overall current design looks good t

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

2016-05-03 Thread Gwen Shapira
Are we planning on updating the security section in Kafka documentation? On Tue, May 3, 2016 at 12:18 AM, Rajini Sivaram wrote: > Magnus, > > Yes, you are absolutely right. I have fixed the wiki page. Thank you for > pointing it out. > > Regards, > > Rajini > > On Mon, May 2, 2016 at 11:41 PM, Ma

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

2016-05-03 Thread Ismael Juma
Hi Gwen, Very good question! There's a PR here: https://github.com/apache/kafka/pull/1232 :) Ismael On Tue, May 3, 2016 at 5:20 PM, Gwen Shapira wrote: > Are we planning on updating the security section in Kafka documentation? > > On Tue, May 3, 2016 at 12:18 AM, Rajini Sivaram > wrote: > >

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

2016-05-03 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268985#comment-15268985 ] Jun Rao commented on KAFKA-3565: [~becket_qin], thanks for the new results. Are the stats

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

2016-05-03 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268984#comment-15268984 ] Gwen Shapira commented on KAFKA-3585: - Try creating a topic with 100 partitions. This

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

2016-05-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15268998#comment-15268998 ] Jiangjie Qin commented on KAFKA-3565: - [~junrao] The tests run above have two differen

[GitHub] kafka pull request: KAFKA-3517: Add documentation for SASL/PLAIN

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

[jira] [Updated] (KAFKA-3517) Document configuration of SASL/PLAIN and multiple mechanisms

2016-05-03 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-3517: --- Resolution: Fixed Fix Version/s: (was: 0.10.1.0) 0.10.0.0 Status: Reso

[GitHub] kafka pull request: MINOR: add logfilename to error message when f...

2016-05-03 Thread norwood
GitHub user norwood opened a pull request: https://github.com/apache/kafka/pull/1307 MINOR: add logfilename to error message when file missing @ewencp You can merge this pull request into a Git repository by running: $ git pull https://github.com/norwood/kafka log-filename-whe

[jira] [Created] (KAFKA-3650) AWS test script fails to install vagrant

2016-05-03 Thread Roger Hoover (JIRA)
Roger Hoover created KAFKA-3650: --- Summary: AWS test script fails to install vagrant Key: KAFKA-3650 URL: https://issues.apache.org/jira/browse/KAFKA-3650 Project: Kafka Issue Type: Bug Affe

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

2016-05-03 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269113#comment-15269113 ] Jun Rao commented on KAFKA-3565: [~becket_qin], thanks for the explanation. The results no

[GitHub] kafka pull request: KAFKA-3650: Fix vagrant download URL

2016-05-03 Thread theduderog
GitHub user theduderog opened a pull request: https://github.com/apache/kafka/pull/1308 KAFKA-3650: Fix vagrant download URL You can merge this pull request into a Git repository by running: $ git pull https://github.com/theduderog/kafka fix_vagrant_dl Alternatively you can r

[jira] [Commented] (KAFKA-3650) AWS test script fails to install vagrant

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

[jira] [Updated] (KAFKA-3650) AWS test script fails to install vagrant

2016-05-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3650?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3650: - Assignee: Roger Hoover > AWS test script fails to install vagrant > --

[jira] [Commented] (KAFKA-3650) AWS test script fails to install vagrant

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

[GitHub] kafka pull request: KAFKA-3650: Fix vagrant download URL

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

[jira] [Resolved] (KAFKA-3650) AWS test script fails to install vagrant

2016-05-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3650?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-3650. -- Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request

[GitHub] kafka pull request: MINOR: add logfilename to error message when f...

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

[jira] [Commented] (KAFKA-3144) report members with no assigned partitions in ConsumerGroupCommand

2016-05-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269145#comment-15269145 ] Vahid Hashemian commented on KAFKA-3144: [~junrao] I haven't heard from [~Konrad K

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-05-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269148#comment-15269148 ] Ewen Cheslack-Postava commented on KAFKA-3209: -- To help clarify [~Skandragon]

[jira] [Commented] (KAFKA-3144) report members with no assigned partitions in ConsumerGroupCommand

2016-05-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269149#comment-15269149 ] Vahid Hashemian commented on KAFKA-3144: Sorry for broken formatting. I hope what

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

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269155#comment-15269155 ] Ismael Juma commented on KAFKA-3646: Updating this issue with conversation that happen

Re: [VOTE] KIP-57: Interoperable LZ4 Framing

2016-05-03 Thread Ewen Cheslack-Postava
+1 One caveat on the vote though -- I don't know the details of LZ4 (format, libraries, etc) well enough to have a handle on whether the changes under "KafkaLZ4* code" are going to be sufficient to get broad support from other LZ4 libraries. Are we going to have multiple implementations we can tes

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

2016-05-03 Thread Soumyajit Sahu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3448?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Soumyajit Sahu updated KAFKA-3448: -- Description: When an address is written textually, the zone index is appended to the address, se

[jira] [Updated] (KAFKA-3448) Support zone index in IPv6 regex

2016-05-03 Thread Soumyajit Sahu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3448?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Soumyajit Sahu updated KAFKA-3448: -- Summary: Support zone index in IPv6 regex (was: IPV6 Regex is missing % character) > Support z

[jira] [Commented] (KAFKA-3642) Fix NPE from ProcessorStateManager when the changelog topic not exists

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

[GitHub] kafka pull request: KAFKA-3642: Fix NPE from ProcessorStateManager...

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

[jira] [Updated] (KAFKA-3642) Fix NPE from ProcessorStateManager when the changelog topic not exists

2016-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3642: - Resolution: Fixed Fix Version/s: (was: 0.10.1.0) 0.10.0.0

[GitHub] kafka pull request: KAFKA-3128; Add metrics for ZooKeeper events z...

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

[jira] [Commented] (KAFKA-3128) Add metrics for ZooKeeper events

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

[jira] [Updated] (KAFKA-3128) Add metrics for ZooKeeper events

2016-05-03 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3128?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-3128: --- Resolution: Fixed Fix Version/s: (was: 0.10.1.0) 0.10.0.0 Status: Reso

Re: [VOTE] KIP-57: Interoperable LZ4 Framing

2016-05-03 Thread Dana Powers
Yes, great point. The intent of adding "naive" support for the remaining LZ4 header flags (contentsize and contentchecksum) is to avoid rejecting any lz4 messages framed according to the interoperable spec (v1.5.1). If we can accept all such messages (which this KIP does), we should never have to m

[GitHub] kafka pull request: KAFKA-3448: Support zone index in IPv6 regex

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

[jira] [Commented] (KAFKA-3448) Support zone index in IPv6 regex

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

[jira] [Updated] (KAFKA-3448) Support zone index in IPv6 regex

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3448?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3448: --- Resolution: Fixed Fix Version/s: (was: 0.10.1.0) 0.10.0.0 Stat

[jira] [Updated] (KAFKA-3479) Add new consumer metrics documentation

2016-05-03 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3479: Assignee: Kaufman Ng > Add new consumer metrics documentation >

Re: [VOTE] KIP-57: Interoperable LZ4 Framing

2016-05-03 Thread Magnus Edenhill
Good work on this Dana. I'll test it with librdkafka (which uses the official liblz4) and report back. 2016-05-03 20:02 GMT+02:00 Dana Powers : > Yes, great point. The intent of adding "naive" support for the > remaining LZ4 header flags (contentsize and contentchecksum) is to > avoid rejecting

[GitHub] kafka pull request: Minor fixes to AWS init script for testing

2016-05-03 Thread theduderog
GitHub user theduderog opened a pull request: https://github.com/apache/kafka/pull/1309 Minor fixes to AWS init script for testing A path was wrong in the script and in the documentation. You can merge this pull request into a Git repository by running: $ git pull https://githu

Re: [DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-05-03 Thread Gwen Shapira
Sorry for the delay: Two questions that we didn't see in the wiki: 1. Is there an expiration for delegation tokens? Renewal? How do we revoke them? 2. If we want to use delegation tokens for "do-as" (say, submit Storm job as my user), we will need a producer for every job (we can't share them betw

[jira] [Updated] (KAFKA-3637) Add method that checks if streams are initialised

2016-05-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-3637: Description: Currently when streams are initialised and started with streams.start(), there is no w

[jira] [Assigned] (KAFKA-3480) Autogenerate metrics documentation

2016-05-03 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3480?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng reassigned KAFKA-3480: -- Assignee: James Cheng > Autogenerate metrics documentation > --

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

2016-05-03 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-3517; Add documentation for SASL/PLAIN -- [...truncated 4065 lines...] kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenReplicas PASSED kafka.a

Build failed in Jenkins: kafka-trunk-jdk7 #1240

2016-05-03 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-3517; Add documentation for SASL/PLAIN [me] KAFKA-3650: Fix vagrant download URL [me] MINOR: add logfilename to error message when file missing -- [...truncated 294

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

2016-05-03 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3650: Fix vagrant download URL [me] MINOR: add logfilename to error message when file missing [wangguoz] KAFKA-3642: Fix NPE from ProcessorStateManager when the changelog topic [junrao] KAFKA-3128; Add metric

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

2016-05-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269491#comment-15269491 ] Jiangjie Qin commented on KAFKA-3565: - [~junrao] Thanks for help looking into this. I

[GitHub] kafka pull request: MINOR: Added more integration tests

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

[jira] [Commented] (KAFKA-3587) LogCleaner fails due to incorrect offset map computation on a replica

2016-05-03 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269586#comment-15269586 ] James Cheng commented on KAFKA-3587: The fix for KAFKA-3587 may resolve both KAFKA-223

[jira] [Created] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" excepion, it should also remove the condition object from the waiters

2016-05-03 Thread Mayuresh Gharat (JIRA)
Mayuresh Gharat created KAFKA-3651: -- Summary: Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" excepion, it should also remove the condition object from the waiters deque Key:

[jira] [Commented] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" excepion, it should also remove the condition object from the waite

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269596#comment-15269596 ] Ismael Juma commented on KAFKA-3651: Maybe we can fix that as part of https://github.c

[GitHub] kafka pull request: KAFKA-3383: remove in flight request only afte...

2016-05-03 Thread zhuchen1018
Github user zhuchen1018 closed the pull request at: https://github.com/apache/kafka/pull/1050 --- 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

Jenkins build is back to normal : kafka-trunk-jdk7 #1241

2016-05-03 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-3383) Producer should not remove an in flight request before successfully parsing the response.

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

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

2016-05-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Added more integration tests -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on jenkins-test-a4 (jenkins-cloud-4GB c

Re: [DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-05-03 Thread Harsha
Gwen, On your second point. Can you describe a usecase where mutliple clients ended up sharing a producer and even if they do why can't they not use single token that producer captures. Why would we need multiple clients with different tokens s

Build failed in Jenkins: kafka-trunk-jdk7 #1242

2016-05-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Added more integration tests -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on jenkins-test-a4 (jenkins-cloud-4GB

[jira] [Commented] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" excepion, it should also remove the condition object from the waite

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269603#comment-15269603 ] Ismael Juma commented on KAFKA-3651: Actually, let's do it separately as we can then a

[jira] [Commented] (KAFKA-3330) Truncate log cleaner offset checkpoint if the log is truncated

2016-05-03 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269671#comment-15269671 ] James Cheng commented on KAFKA-3330: We ran into this today. While we are waiting for

[jira] [Created] (KAFKA-3652) Allow only one ApiVersionsRequest before SASL handshake

2016-05-03 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-3652: - Summary: Allow only one ApiVersionsRequest before SASL handshake Key: KAFKA-3652 URL: https://issues.apache.org/jira/browse/KAFKA-3652 Project: Kafka Issue

[jira] [Commented] (KAFKA-3652) Allow only one ApiVersionsRequest before SASL handshake

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

[GitHub] kafka pull request: KAFKA-3652: Allow atmost one ApiVersionsReques...

2016-05-03 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/1310 KAFKA-3652: Allow atmost one ApiVersionsRequest before SASL handshake Additional check and unit test. You can merge this pull request into a Git repository by running: $ git pull https://

[jira] [Updated] (KAFKA-3639) Configure default serdes passed via StreamsConfig

2016-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3639?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3639: - Fix Version/s: (was: 0.10.1.0) 0.10.0.0 > Configure default serdes passed v

[jira] [Assigned] (KAFKA-3639) Configure default serdes passed via StreamsConfig

2016-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3639?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-3639: Assignee: Guozhang Wang > Configure default serdes passed via StreamsConfig > -

[GitHub] kafka pull request: KAFKA-3639: Configure default serdes upon cons...

2016-05-03 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/1311 KAFKA-3639: Configure default serdes upon construction You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K3639 Alternativ

[jira] [Commented] (KAFKA-3639) Configure default serdes passed via StreamsConfig

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

[jira] [Created] (KAFKA-3653) expose the queue size in ControllerChannelManager

2016-05-03 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-3653: -- Summary: expose the queue size in ControllerChannelManager Key: KAFKA-3653 URL: https://issues.apache.org/jira/browse/KAFKA-3653 Project: Kafka Issue Type: Improvement

[jira] [Updated] (KAFKA-3565) bceunbfjcvtdvidckvjrrcklvtdnjgvlj

2016-05-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-3565: Summary: bceunbfjcvtdvidckvjrrcklvtdnjgvlj (was: Producer's throughput lower with compressed data a

[jira] [Assigned] (KAFKA-3653) expose the queue size in ControllerChannelManager

2016-05-03 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3653?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira reassigned KAFKA-3653: --- Assignee: Gwen Shapira > expose the queue size in ControllerChannelManager >

[GitHub] kafka pull request: KAFKA-3632: remove fetcher metrics on shutdown...

2016-05-03 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1312 KAFKA-3632: remove fetcher metrics on shutdown and leader migration You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-3632

[jira] [Assigned] (KAFKA-3632) ConsumerLag metrics persist after partition migration

2016-05-03 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3632?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-3632: -- Assignee: Jason Gustafson > ConsumerLag metrics persist after partition migration > ---

[jira] [Commented] (KAFKA-3632) ConsumerLag metrics persist after partition migration

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

[GitHub] kafka pull request: MINOR: Clean up of ConsumerCoordinator and Par...

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

[jira] [Updated] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" exception, it should also remove the condition object from the waiter

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3651?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3651: --- Summary: Whenever the BufferPool throws a "Failed to allocate memory within the configured max blockin

[jira] [Commented] (KAFKA-3565) bceunbfjcvtdvidckvjrrcklvtdnjgvlj

2016-05-03 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269805#comment-15269805 ] Jun Rao commented on KAFKA-3565: [~becket_qin], thanks for the new results. Could you also

[GitHub] kafka pull request: KAFKA-3648; maxTimeToBlock in BufferPool.alloc...

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

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

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

Jenkins build is back to normal : kafka-trunk-jdk8 #580

2016-05-03 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-3648) maxTimeToBlock in BufferPool.allocate should be enforced

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3648?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3648: --- Summary: maxTimeToBlock in BufferPool.allocate should be enforced (was: maxTimeToBlock should be enfo

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

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3648?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3648. Resolution: Fixed Fix Version/s: 0.10.0.0 > maxTimeToBlock should be enforced >

[jira] [Commented] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" exception, it should also remove the condition object from the wait

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269821#comment-15269821 ] Ismael Juma commented on KAFKA-3651: [~mgharat], are you planning to submit a PR for t

[jira] [Commented] (KAFKA-3173) Error while moving some partitions to OnlinePartition state

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3173?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269832#comment-15269832 ] Ismael Juma commented on KAFKA-3173: [~gwenshap], the PR has been ready for a while, w

[jira] [Updated] (KAFKA-3632) ConsumerLag metrics persist after partition migration

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3632?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3632: --- Reviewer: Ismael Juma Status: Patch Available (was: Open) > ConsumerLag metrics persist after p

[jira] [Commented] (KAFKA-3565) bceunbfjcvtdvidckvjrrcklvtdnjgvlj

2016-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269836#comment-15269836 ] Guozhang Wang commented on KAFKA-3565: -- [~becket_qin] JIRA title? :) > bceunbfjcvtdv

[jira] [Commented] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" exception, it should also remove the condition object from the wait

2016-05-03 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269837#comment-15269837 ] Mayuresh Gharat commented on KAFKA-3651: [~ijuma] Yes just working on the unit tes

[jira] [Updated] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" exception, it should also remove the condition object from the waiter

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3651?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3651: --- Fix Version/s: 0.10.0.0 > Whenever the BufferPool throws a "Failed to allocate memory within the > co

[jira] [Commented] (KAFKA-3651) Whenever the BufferPool throws a "Failed to allocate memory within the configured max blocking time" exception, it should also remove the condition object from the wait

2016-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15269841#comment-15269841 ] Ismael Juma commented on KAFKA-3651: Great, make sure to pull from trunk as the other

[jira] [Created] (KAFKA-3654) ConnectorConfig defs short circuit implementation config checks

2016-05-03 Thread Liquan Pei (JIRA)
Liquan Pei created KAFKA-3654: - Summary: ConnectorConfig defs short circuit implementation config checks Key: KAFKA-3654 URL: https://issues.apache.org/jira/browse/KAFKA-3654 Project: Kafka Issu

  1   2   >