Re: [VOTE] KIP-74: Add FetchResponse size limit in bytes

2016-08-18 Thread Manikumar Reddy
+1 (non-binding) This feature help us control memory footprint and allows consumer to progress on fetching large messages. On Fri, Aug 19, 2016 at 10:32 AM, Gwen Shapira wrote: > +1 (binding) > > On Thu, Aug 18, 2016 at 1:47 PM, Andrey L. Neporada >

Re: [VOTE] KIP-74: Add FetchResponse size limit in bytes

2016-08-18 Thread Gwen Shapira
+1 (binding) On Thu, Aug 18, 2016 at 1:47 PM, Andrey L. Neporada wrote: > Hi all! > I’ve modified KIP-74 a little bit (as requested by Jason Gustafson & Jun Rao): > 1) provided more detailed explanation on memory usage (no functional changes) > 2) renamed

Re: [VOTE] KIP-74: Add FetchResponse size limit in bytes

2016-08-18 Thread Jason Gustafson
+1 (non-binding) Thanks Andrey! On Thu, Aug 18, 2016 at 1:47 PM, Andrey L. Neporada < anepor...@yandex-team.ru> wrote: > Hi all! > I’ve modified KIP-74 a little bit (as requested by Jason Gustafson & Jun > Rao): > 1) provided more detailed explanation on memory usage (no functional > changes) >

[jira] [Resolved] (KAFKA-4063) Add support for infinite endpoints for range queries in Kafka Streams KV stores

2016-08-18 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4063?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy resolved KAFKA-4063. Resolution: Duplicate > Add support for infinite endpoints for range queries in Kafka

[GitHub] kafka pull request #1762: KAFKA-3949: Fix race condition when metadata updat...

2016-08-18 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1762 KAFKA-3949: Fix race condition when metadata update arrives during rebalance You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka

[jira] [Commented] (KAFKA-3949) Consumer topic subscription change may be ignored if a rebalance is in progress

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

[jira] [Updated] (KAFKA-3845) Support per-connector converters

2016-08-18 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3845: Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-3845) Support per-connector converters

2016-08-18 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15427571#comment-15427571 ] ASF GitHub Bot commented on KAFKA-3845: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #1721: KAFKA-3845: KIP-75: Add per-connector converters

2016-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1721 --- 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 #1740: MINOR: Remove # from .bat start script

2016-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1740 --- 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 #1732: MINOR: Clarification in producer config documentat...

2016-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1732 --- 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] [Commented] (KAFKA-4051) Strange behavior during rebalance when turning the OS clock back

2016-08-18 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15427554#comment-15427554 ] Gwen Shapira commented on KAFKA-4051: - I was definitely concerned about complete replacement

Build failed in Jenkins: kafka-0.10.0-jdk7 #190

2016-08-18 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4037: Make Connect REST API retries aware of 409 CONFLICT errors -- [...truncated 6449 lines...] org.apache.kafka.connect.runtime.WorkerSourceTaskTest > testCommit

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

2016-08-18 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4037: Make Connect REST API retries aware of 409 CONFLICT errors -- [...truncated 11979 lines...] org.apache.kafka.streams.processor.internals.MinTimestampTrackerTest >

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

2016-08-18 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4037: Make Connect REST API retries aware of 409 CONFLICT errors -- [...truncated 11997 lines...] org.apache.kafka.streams.kstream.internals.KTableKTableLeftJoinTest >

[GitHub] kafka pull request #1761: KAFKA-4064 Add support for infinite endpoints for ...

2016-08-18 Thread theduderog
GitHub user theduderog opened a pull request: https://github.com/apache/kafka/pull/1761 KAFKA-4064 Add support for infinite endpoints for range queries in Kafka Streams KV stores @guozhangwang I had to fix the bug with in-memory ranges being exclusive and RocksDB being

[jira] [Commented] (KAFKA-4064) Add support for infinite endpoints for range queries in Kafka Streams KV stores

2016-08-18 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4064?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15427374#comment-15427374 ] ASF GitHub Bot commented on KAFKA-4064: --- GitHub user theduderog opened a pull request:

[jira] [Created] (KAFKA-4063) Add support for infinite endpoints for range queries in Kafka Streams KV stores

2016-08-18 Thread Roger Hoover (JIRA)
Roger Hoover created KAFKA-4063: --- Summary: Add support for infinite endpoints for range queries in Kafka Streams KV stores Key: KAFKA-4063 URL: https://issues.apache.org/jira/browse/KAFKA-4063 Project:

[jira] [Created] (KAFKA-4064) Add support for infinite endpoints for range queries in Kafka Streams KV stores

2016-08-18 Thread Roger Hoover (JIRA)
Roger Hoover created KAFKA-4064: --- Summary: Add support for infinite endpoints for range queries in Kafka Streams KV stores Key: KAFKA-4064 URL: https://issues.apache.org/jira/browse/KAFKA-4064 Project:

[jira] [Resolved] (KAFKA-3054) Connect Herder fail forever if sent a wrong connector config or task config

2016-08-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-3054. Resolution: Done > Connect Herder fail forever if sent a wrong connector config or task

[jira] [Commented] (KAFKA-3054) Connect Herder fail forever if sent a wrong connector config or task config

2016-08-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3054?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15427362#comment-15427362 ] Shikhar Bhushan commented on KAFKA-3054: Addressing this in KAFKA-4042, which should take care of

[jira] [Commented] (KAFKA-3438) Rack Aware Replica Reassignment should warn of overloaded brokers

2016-08-18 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15427361#comment-15427361 ] Vahid Hashemian commented on KAFKA-3438: [~benstopford] Are you working on this? If not, I can

[jira] [Commented] (KAFKA-4037) Transient failure in ConnectRestApiTest

2016-08-18 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4037?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15427284#comment-15427284 ] ASF GitHub Bot commented on KAFKA-4037: --- Github user asfgit closed the pull request at:

[jira] [Updated] (KAFKA-4037) Transient failure in ConnectRestApiTest

2016-08-18 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4037?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4037: - Resolution: Fixed Fix Version/s: 0.10.1.0 0.10.0.2

[GitHub] kafka pull request #1733: KAFKA-4037: Make Connect REST API retries aware of...

2016-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1733 --- 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: [VOTE] KIP-74: Add FetchResponse size limit in bytes

2016-08-18 Thread Andrey L. Neporada
Hi all! I’ve modified KIP-74 a little bit (as requested by Jason Gustafson & Jun Rao): 1) provided more detailed explanation on memory usage (no functional changes) 2) renamed “fetch.response.max.bytes” -> “fetch.max.bytes” Let’s continue voting in this thread. Thanks! Andrey. > On 17 Aug 2016,

[jira] [Commented] (KAFKA-2170) 10 LogTest cases failed for file.renameTo failed under windows

2016-08-18 Thread Soumyajit Sahu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15427096#comment-15427096 ] Soumyajit Sahu commented on KAFKA-2170: --- [~haraldk] Thanks for checking it out. I didn't try with

[jira] [Created] (KAFKA-4062) Require --print-data-log if --offsets-decoder is enabled for DumpLogOffsets

2016-08-18 Thread Dustin Cote (JIRA)
Dustin Cote created KAFKA-4062: -- Summary: Require --print-data-log if --offsets-decoder is enabled for DumpLogOffsets Key: KAFKA-4062 URL: https://issues.apache.org/jira/browse/KAFKA-4062 Project: Kafka

OffsetOutOfRange errors during broker leader transitions

2016-08-18 Thread Bill Warshaw
We are running a 3-node deployment of Kafka, and on several of our testing sites we have seen the following scenario occur: - "auto.offset.reset" is set to "earliest" - A client is reading from Kafka, and at some point the broker throws an OffsetOutOfRangeException, causing the consumer to seek

[jira] [Assigned] (KAFKA-3949) Consumer topic subscription change may be ignored if a rebalance is in progress

2016-08-18 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-3949: -- Assignee: Jason Gustafson > Consumer topic subscription change may be ignored if a

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Gwen Shapira
Yes, I think its a great discussion to have. There are definitely pros and cons to both approaches and worth thinking about the right way forward. On Thu, Aug 18, 2016 at 11:03 AM, Todd Palino wrote: > This all makes a lot of sense, and mirrors what I’m thinking as I finally

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Harsha Chintalapani
Congrats Gwen. -Harsha On Thu, Aug 18, 2016 at 10:59 AM Mayuresh Gharat wrote: > Congrats Gwen :) > > Thanks, > > Mayuresh > > On Thu, Aug 18, 2016 at 10:27 AM, Gwen Shapira wrote: > > > Thanks team Kafka :) Very excited and happy to contribute

Re: Should we have a KIP call?

2016-08-18 Thread Jun Rao
Grant, That sounds like a good idea. I will send out an invite for this Tue at 11am. There are quite a few KIPs in your list and we probably can't cover them all in one call. Perhaps we can do a quick status check on those that have been voted - KIP-33: Add a time based log index - KIP-50:

Re: [VOTE] KIP-75 - Add per-connector Converters

2016-08-18 Thread Ewen Cheslack-Postava
Thanks everyone, this passes with 6 binding, 2 non-binding +1s. -Ewen On Mon, Aug 15, 2016 at 5:25 PM, Guozhang Wang wrote: > +1. > > On Mon, Aug 15, 2016 at 2:55 PM, Jason Gustafson > wrote: > > > +1 (non-binding) > > > > On Mon, Aug 15, 2016 at 2:53

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Todd Palino
Congratulations, Gwen! Well deserved :) -Todd On Thu, Aug 18, 2016 at 10:27 AM, Gwen Shapira wrote: > Thanks team Kafka :) Very excited and happy to contribute and be part > of this fantastic community. > > > > On Thu, Aug 18, 2016 at 9:52 AM, Guozhang Wang

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Todd Palino
Yeah, I’m good with where we are right now on this KIP. It’s a workable solution that we can add tooling to support. I would prefer to have soft quotas, but given that it is more complex to implement, we can go with hard quotas for the time being and consider it as an improvement later. -Todd

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Jun Rao
Todd, Thanks for the detailed reply. So, it sounds like that you are ok with the current proposal in the KIP for now and we can brainstorm on more automated stuff separately? Are you comfortable with starting the vote on the current proposal? Jun On Thu, Aug 18, 2016 at 11:00 AM, Todd Palino

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Todd Palino
This all makes a lot of sense, and mirrors what I’m thinking as I finally took some time to really walk through scenarios around why we move partitions around. What I’m wondering if it makes sense to have a conversation around breaking out the controller entirely, separating it from the brokers,

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Todd Palino
Joel just reminded me to take another look at this one :) So first off, this is great. It’s something that we definitely need to have, especially as we get into the realm of moving partitions around more often. I do prefer to have the cluster handle this automatically. What I envision is a single

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Mayuresh Gharat
Congrats Gwen :) Thanks, Mayuresh On Thu, Aug 18, 2016 at 10:27 AM, Gwen Shapira wrote: > Thanks team Kafka :) Very excited and happy to contribute and be part > of this fantastic community. > > > > On Thu, Aug 18, 2016 at 9:52 AM, Guozhang Wang wrote:

Should we have a KIP call?

2016-08-18 Thread Grant Henke
I am thinking it might be a good time to have a Kafka KIP call. There are a lot of KIPs and discussions in progress that could benefit from a "quick" call to discuss, coordinate, and prioritize. Some of the voted topics we could discuss are: (I didn't include ones that were just voted or will

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Gwen Shapira
Just my take, since Jun and Ben originally wanted to solve a more general approach and I talked them out of it :) When we first add the feature, safety is probably most important in getting people to adopt it - I wanted to make the feature very safe by never throttling something admins don't want

[jira] [Commented] (KAFKA-3937) Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages

2016-08-18 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426851#comment-15426851 ] ASF GitHub Bot commented on KAFKA-3937: --- GitHub user wiyu opened a pull request:

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Jun Rao
Joel, Yes, for your second comment. The tricky thing is still to figure out which replicas to throttle and by how much since in general, admins probably don't want already in-sync or close to in-sync replicas to be throttled. It would be great to get Todd's opinion on this. Could you ping him?

[GitHub] kafka pull request #1760: KAFKA-3937: Kafka Clients Leak Native Memory For L...

2016-08-18 Thread wiyu
GitHub user wiyu opened a pull request: https://github.com/apache/kafka/pull/1760 KAFKA-3937: Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages @ijuma - Making the change against trunk based on your suggestions to have the stream closing handled in

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Gwen Shapira
Thanks team Kafka :) Very excited and happy to contribute and be part of this fantastic community. On Thu, Aug 18, 2016 at 9:52 AM, Guozhang Wang wrote: > Congrats Gwen! > > On Thu, Aug 18, 2016 at 9:27 AM, Ashish Singh wrote: > >> Congrats Gwen! >> >>

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Joel Koshy
> For your first comment. We thought about determining "effect" replicas > automatically as well. First, there are some tricky stuff that one has to > Auto-detection of effect traffic: i'm fairly certain it's doable but definitely tricky. I'm also not sure it is something worth tackling at the

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-18 Thread Jun Rao
Joel, For your first comment. We thought about determining "effect" replicas automatically as well. First, there are some tricky stuff that one has to figure out as Ben pointed out. For example, what's the definition of "effect" replicas. If a replica falls out of ISR temporarily, does it become

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Guozhang Wang
Congrats Gwen! On Thu, Aug 18, 2016 at 9:27 AM, Ashish Singh wrote: > Congrats Gwen! > > On Thursday, August 18, 2016, Grant Henke wrote: > > > Congratulations Gwen! > > > > > > > > On Thu, Aug 18, 2016 at 2:36 AM, Ismael Juma >

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Vahid S Hashemian
Congratulations Gwen! --Vahid From: Ashish Singh To: "dev@kafka.apache.org" Date: 08/18/2016 09:27 AM Subject:Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira Congrats Gwen! On Thursday, August 18, 2016, Grant Henke

Re: [DISCUSS] KIP-4 ACL Admin Schema

2016-08-18 Thread Grant Henke
Thanks for the feedback. Below are some responses: > I don't have any problem with breaking things into 2 requests if it's > necessary or optimal. But can you explain why separate requests "vastly > simplifies the broker side implementation"? It doesn't seem like it should > be particularly

[jira] [Assigned] (KAFKA-4007) Improve fetch pipelining for low values of max.poll.records

2016-08-18 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4007?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison reassigned KAFKA-4007: - Assignee: Mickael Maison > Improve fetch pipelining for low values of max.poll.records >

Re: [VOTE] KIP-74: Add Fetch Response Size Limit in Bytes

2016-08-18 Thread Andrey L. Neporada
Hi, Jason! > On 17 Aug 2016, at 21:53, Jason Gustafson wrote: > > Hi Andrey, > > Thanks for picking this up and apologies for the late comment. > > One thing worth mentioning is that the consumer actually sends multiple > parallel fetch requests, one for each broker that

[jira] [Commented] (KAFKA-4056) Kafka logs values of sensitive configs like passwords

2016-08-18 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426603#comment-15426603 ] Mickael Maison commented on KAFKA-4056: --- I've sent a PR to only log the name is case of unknown

[jira] [Commented] (KAFKA-4056) Kafka logs values of sensitive configs like passwords

2016-08-18 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426600#comment-15426600 ] ASF GitHub Bot commented on KAFKA-4056: --- GitHub user mimaison opened a pull request:

[GitHub] kafka pull request #1759: KAFKA-4056: Kafka logs values of sensitive configs...

2016-08-18 Thread mimaison
GitHub user mimaison opened a pull request: https://github.com/apache/kafka/pull/1759 KAFKA-4056: Kafka logs values of sensitive configs like passwords In case of unknown configs, only list the name without the value You can merge this pull request into a Git repository by running:

[jira] [Updated] (KAFKA-4039) Exit Strategy: using exceptions instead of inline invocation of exit/halt

2016-08-18 Thread Alexey Ozeritskiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alexey Ozeritskiy updated KAFKA-4039: - Attachment: deadlock-stack2 > Exit Strategy: using exceptions instead of inline

[jira] [Commented] (KAFKA-4056) Kafka logs values of sensitive configs like passwords

2016-08-18 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426576#comment-15426576 ] Mickael Maison commented on KAFKA-4056: --- This is what I've been trying :) Connecting to a broker on

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Grant Henke
Congratulations Gwen! On Thu, Aug 18, 2016 at 2:36 AM, Ismael Juma wrote: > Congratulations Gwen! Great news. > > Ismael > > On 18 Aug 2016 2:44 am, "Jun Rao" wrote: > > > Hi, Everyone, > > > > Gwen Shapira has been active in the Kafka community since

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

2016-08-18 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4056) Kafka logs values of sensitive configs like passwords

2016-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426541#comment-15426541 ] Ismael Juma commented on KAFKA-4056: [~mimaison], the key thing to trigger this issue is to set the

[jira] [Commented] (KAFKA-4056) Kafka logs values of sensitive configs like passwords

2016-08-18 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426538#comment-15426538 ] jaikiran pai commented on KAFKA-4056: - [~mimaison], we ran into this on our system when we created a

[jira] [Commented] (KAFKA-4056) Kafka logs values of sensitive configs like passwords

2016-08-18 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426521#comment-15426521 ] Mickael Maison commented on KAFKA-4056: --- Using trunk, I don't see known but unused configurations

Re: [VOTE] KIP:71 Enable log compaction and deletion to co-exist

2016-08-18 Thread Damian Guy
The vote is now concluded and KIP-71 has been accepted. Thanks to everyone for your input. Regards, Damian On Tue, 16 Aug 2016 at 01:25 Joel Koshy wrote: > +1 > > On Mon, Aug 15, 2016 at 4:58 PM, Ewen Cheslack-Postava > wrote: > > > +1 (binding) > > >

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

2016-08-18 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4038; Transient failure in -- [...truncated 12008 lines...] org.apache.kafka.streams.kstream.internals.KTableKTableLeftJoinTest > testJoin PASSED

[jira] [Created] (KAFKA-4061) Apache Kafka failover is not working

2016-08-18 Thread Sebastian Bruckner (JIRA)
Sebastian Bruckner created KAFKA-4061: - Summary: Apache Kafka failover is not working Key: KAFKA-4061 URL: https://issues.apache.org/jira/browse/KAFKA-4061 Project: Kafka Issue Type: Bug

[jira] [Comment Edited] (KAFKA-3875) Transient test failure: kafka.api.SslProducerSendTest.testSendNonCompressedMessageWithCreateTime

2016-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3875?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426268#comment-15426268 ] Ismael Juma edited comment on KAFKA-3875 at 8/18/16 11:52 AM: -- [~junrao], we

[jira] [Updated] (KAFKA-4038) Transient failure in DeleteTopicsRequestTest.testErrorDeleteTopicRequests

2016-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4038?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4038: --- Resolution: Fixed Fix Version/s: 0.10.1.0 Status: Resolved (was: Patch Available)

[jira] [Commented] (KAFKA-4038) Transient failure in DeleteTopicsRequestTest.testErrorDeleteTopicRequests

2016-08-18 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426295#comment-15426295 ] ASF GitHub Bot commented on KAFKA-4038: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #1737: KAFKA-4038: Transient failure in DeleteTopicsReque...

2016-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1737 --- 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] [Commented] (KAFKA-4051) Strange behavior during rebalance when turning the OS clock back

2016-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426292#comment-15426292 ] Ismael Juma commented on KAFKA-4051: [~rsivaram], option 1 seems worth a try. Note that it would have

[jira] [Commented] (KAFKA-4051) Strange behavior during rebalance when turning the OS clock back

2016-08-18 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426290#comment-15426290 ] Rajini Sivaram commented on KAFKA-4051: --- [~ijuma] As you have pointed out, there are inevitably

[jira] [Commented] (KAFKA-2170) 10 LogTest cases failed for file.renameTo failed under windows

2016-08-18 Thread Harald Kirsch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426278#comment-15426278 ] Harald Kirsch commented on KAFKA-2170: -- The patch does not seem to work completely for compaction. I

[jira] [Commented] (KAFKA-3875) Transient test failure: kafka.api.SslProducerSendTest.testSendNonCompressedMessageWithCreateTime

2016-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3875?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426268#comment-15426268 ] Ismael Juma commented on KAFKA-3875: [~junrao], we had another failure for this test: {code}

[jira] [Reopened] (KAFKA-3875) Transient test failure: kafka.api.SslProducerSendTest.testSendNonCompressedMessageWithCreateTime

2016-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3875?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reopened KAFKA-3875: > Transient test failure: > kafka.api.SslProducerSendTest.testSendNonCompressedMessageWithCreateTime >

[jira] [Commented] (KAFKA-4051) Strange behavior during rebalance when turning the OS clock back

2016-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426256#comment-15426256 ] Ismael Juma commented on KAFKA-4051: As discussed in the thread, Kafka uses System.currentTimeMillis()

[jira] [Assigned] (KAFKA-4051) Strange behavior during rebalance when turning the OS clock back

2016-08-18 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reassigned KAFKA-4051: - Assignee: Rajini Sivaram > Strange behavior during rebalance when turning the OS clock

[jira] [Resolved] (KAFKA-4045) Investigate feasibility of hooking into RocksDb's cache

2016-08-18 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4045?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-4045. - Resolution: Fixed > Investigate feasibility of hooking into RocksDb's cache >

[GitHub] kafka pull request #1758: KAFKA-4019: Update log cleaner to handle max messa...

2016-08-18 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/1758 KAFKA-4019: Update log cleaner to handle max message size of topics Grow read and write buffers of cleaner up to the maximum message size of the log being cleaned if the topic has larger max

[jira] [Commented] (KAFKA-4019) LogCleaner should grow read/write buffer to max message size for the topic

2016-08-18 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4019?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15426092#comment-15426092 ] ASF GitHub Bot commented on KAFKA-4019: --- GitHub user rajinisivaram opened a pull request:

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-18 Thread Ismael Juma
Congratulations Gwen! Great news. Ismael On 18 Aug 2016 2:44 am, "Jun Rao" wrote: > Hi, Everyone, > > Gwen Shapira has been active in the Kafka community since she became a > Kafka committer > about a year ago. I am glad to announce that Gwen is now a member of Kafka > PMC.

Re: [jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-08-18 Thread Andrew Coates
Hi [~junrao], has there been any more discussion or progress on this issue? Thanks, Andy On Tue, 12 Jul 2016 at 10:11, Andy Coates (JIRA) wrote: > > [ >

[jira] [Commented] (KAFKA-3172) Consumer threads stay in 'Watiting' status and are blocked at consumer poll method

2016-08-18 Thread Oleg Gorobets (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425986#comment-15425986 ] Oleg Gorobets commented on KAFKA-3172: -- Couple more observations: (1) this happens only when I am

[jira] [Resolved] (KAFKA-2199) Make signing artifacts optional, setting maven repository possible from command line

2016-08-18 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2199?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy resolved KAFKA-2199. Resolution: Fixed > Make signing artifacts optional, setting maven repository possible from