[jira] [Updated] (KAFKA-4459) rat license check not running in Jenkins

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4459: --- Fix Version/s: 0.10.2.0 > rat license check not running in Jenkins > -

[jira] [Created] (KAFKA-4459) rat license check not running in Jenkins

2016-11-29 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-4459: -- Summary: rat license check not running in Jenkins Key: KAFKA-4459 URL: https://issues.apache.org/jira/browse/KAFKA-4459 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #2187: Revert "KAFKA-4345; Run decktape test for each pul...

2016-11-29 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2187 Revert "KAFKA-4345; Run decktape test for each pull request" This reverts commit e035fc039598127e88f31739458f705290b1fdba for the following reasons: 1. License files are missing causing lo

[jira] [Commented] (KAFKA-4345) Run decktape test for each pull request

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

[jira] [Commented] (KAFKA-4450) Add missing 0.10.1.x upgrade tests and ensure ongoing compatibility checks

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4450?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15704851#comment-15704851 ] Ismael Juma commented on KAFKA-4450: It would be good to understand what [~ewencp] mea

[VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-11-29 Thread Michael Pearce
Hi All, We have been discussing in the below thread and final changes have been made to the KIP wiki based on these discussions. We would now like to put to the vote the following KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-87+-+Add+Compaction+Tombstone+Flag This kip is for havin

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-11-29 Thread Michael Pearce
I assume, that after a period of a week, that there is no concerns now with points 1, and 2 and now we have agreement that headers are useful and needed in Kafka. As such if put to a KIP vote, this wouldn’t be a reason to reject. @ Ignacio on point 4). I think for purpose of getting this KIP mov

[jira] [Updated] (KAFKA-4455) CommitFailedException during rebalance doesn't release resources in processors

2016-11-29 Thread Davor Poldrugo (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Davor Poldrugo updated KAFKA-4455: -- Summary: CommitFailedException during rebalance doesn't release resources in processors (was: C

[jira] [Updated] (KAFKA-4455) CommitFailedException during rebalance doesn't release resources in tasks/processors

2016-11-29 Thread Davor Poldrugo (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Davor Poldrugo updated KAFKA-4455: -- Summary: CommitFailedException during rebalance doesn't release resources in tasks/processors (

Re: [DISCUSS] 0.10.1.1 Plan

2016-11-29 Thread Ismael Juma
On Sat, Nov 26, 2016 at 11:08 PM, radai wrote: > "compatibility guarantees that are expected by people who subclass these > classes" > > sorry if this is not the best thread for this discussion, but I just wanted > to pop in and say that since any subclassing of these will obviously not be > done

Re: [VOTE] KIP-85: Dynamic JAAS configuration for Kafka clients

2016-11-29 Thread Ismael Juma
One more thing: we are using the PASSWORD config type to avoid exposing passwords. This will also make it harder to debug issues with the JAAS config though, it would be good to mention this drawback in the KIP. Ismael On Mon, Nov 28, 2016 at 1:00 PM, Ismael Juma wrote: > I'm very late to this,

[jira] [Updated] (KAFKA-3452) Support session windows

2016-11-29 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-3452: -- Status: Patch Available (was: In Progress) > Support session windows > --- > >

[GitHub] kafka pull request #2188: KAFKA-4455; ensure tasks are closed after CommitFa...

2016-11-29 Thread dpoldrugo
GitHub user dpoldrugo opened a pull request: https://github.com/apache/kafka/pull/2188 KAFKA-4455; ensure tasks are closed after CommitFailedException Details can be found in the task: [KAFKA-4455](https://issues.apache.org/jira/browse/KAFKA-4455) Some discussion was also h

[jira] [Commented] (KAFKA-4455) CommitFailedException during rebalance doesn't release resources in tasks/processors

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

[jira] [Commented] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

2016-11-29 Thread Abhi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705267#comment-15705267 ] Abhi commented on KAFKA-1194: - [~soumyajitsahu] I tried the build that you have shared on one

[jira] [Commented] (KAFKA-4419) Unable to GetOffset when the ACL of topic is defined

2016-11-29 Thread Mohammed amine GARMES (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705265#comment-15705265 ] Mohammed amine GARMES commented on KAFKA-4419: -- hello [~rsivaram] I added a n

[jira] [Updated] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

2016-11-29 Thread Abhi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1194?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Abhi updated KAFKA-1194: Attachment: Untitled.jpg > The kafka broker cannot delete the old log files after the configured time >

[jira] [Comment Edited] (KAFKA-4419) Unable to GetOffset when the ACL of topic is defined

2016-11-29 Thread Mohammed amine GARMES (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705265#comment-15705265 ] Mohammed amine GARMES edited comment on KAFKA-4419 at 11/29/16 1:11 PM:

Re: [DISCUSS] KIP-96 - Add per partition metrics for in-sync and assigned replica count

2016-11-29 Thread Ismael Juma
Hi Xavier, Thanks for the KIP. Sounds good to me. Ismael On Tue, Nov 29, 2016 at 12:40 AM, Xavier Léauté wrote: > Hi, > > I created KIP-96 to propose per partition in-sync / assigned replica > metrics. Should be straightforward, but submitting it for proposal since we > require it for metrics

[jira] [Commented] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705475#comment-15705475 ] Eno Thereska commented on KAFKA-4405: - [~guozhang], [~hachikuji] Indeed I can verify t

[jira] [Created] (KAFKA-4460) Consumer stops getting messages when partition leader dies

2016-11-29 Thread Bernhard Bonigl (JIRA)
Bernhard Bonigl created KAFKA-4460: -- Summary: Consumer stops getting messages when partition leader dies Key: KAFKA-4460 URL: https://issues.apache.org/jira/browse/KAFKA-4460 Project: Kafka

[jira] [Created] (KAFKA-4461) When using ProcessorTopologyTestDriver, the combination of map and .groupByKey does not produce any result

2016-11-29 Thread Hamidreza Afzali (JIRA)
Hamidreza Afzali created KAFKA-4461: --- Summary: When using ProcessorTopologyTestDriver, the combination of map and .groupByKey does not produce any result Key: KAFKA-4461 URL: https://issues.apache.org/jira/brows

[jira] [Commented] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705594#comment-15705594 ] Ismael Juma commented on KAFKA-4405: [~enothereska], do you mean the following? {code

[jira] [Comment Edited] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705594#comment-15705594 ] Ismael Juma edited comment on KAFKA-4405 at 11/29/16 3:26 PM: --

[jira] [Commented] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705655#comment-15705655 ] Eno Thereska commented on KAFKA-4405: - [~ijuma] yes. > Kafka consumer improperly send

[GitHub] kafka pull request #2135: KAFKA-3637: Added initial states

2016-11-29 Thread enothereska
GitHub user enothereska reopened a pull request: https://github.com/apache/kafka/pull/2135 KAFKA-3637: Added initial states You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka KAFKA-3637-streams-state Alternatively

[GitHub] kafka pull request #2135: KAFKA-3637: Added initial states

2016-11-29 Thread enothereska
Github user enothereska closed the pull request at: https://github.com/apache/kafka/pull/2135 --- 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-3637) Add method that checks if streams are initialised

2016-11-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705666#comment-15705666 ] ASF GitHub Bot commented on KAFKA-3637: --- GitHub user enothereska reopened a pull req

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

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

[jira] [Comment Edited] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705475#comment-15705475 ] Eno Thereska edited comment on KAFKA-4405 at 11/29/16 4:02 PM: -

[jira] [Commented] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705691#comment-15705691 ] Ismael Juma commented on KAFKA-4405: [~ysysberserk], are you really seeing the behavio

[jira] [Comment Edited] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705691#comment-15705691 ] Ismael Juma edited comment on KAFKA-4405 at 11/29/16 4:04 PM: --

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-11-29 Thread gharatmayuresh15
+1 (non-binding) Thanks, Mayuresh > On Nov 29, 2016, at 3:18 AM, Michael Pearce wrote: > > Hi All, > > We have been discussing in the below thread and final changes have been made > to the KIP wiki based on these discussions. > > We would now like to put to the vote the following KIP: > ht

Re: [VOTE] KIP-85: Dynamic JAAS configuration for Kafka clients

2016-11-29 Thread Rajini Sivaram
I have added this to the KIP. Thanks, Rajini On Tue, Nov 29, 2016 at 12:14 PM, Ismael Juma wrote: > One more thing: we are using the PASSWORD config type to avoid exposing > passwords. This will also make it harder to debug issues with the JAAS > config though, it would be good to mention this

[jira] [Commented] (KAFKA-4419) Unable to GetOffset when the ACL of topic is defined

2016-11-29 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705720#comment-15705720 ] Rajini Sivaram commented on KAFKA-4419: --- There is an open JIRA to fix this: KAFKA-33

Re: [DISCUSS] 0.10.1.1 Plan

2016-11-29 Thread radai
designing kafka code for stable extensibility is a worthy and noble cause. however, seeing as there are no such derivatives out in the wild yet i think investing the effort right now is a bit premature from kafka's pov. I think its enough simply not to purposefully prevent such extensions. On Tue,

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-11-29 Thread radai
+1 (non-binding) On Tue, Nov 29, 2016 at 8:08 AM, wrote: > +1 (non-binding) > > Thanks, > > Mayuresh > > > > On Nov 29, 2016, at 3:18 AM, Michael Pearce > wrote: > > > > Hi All, > > > > We have been discussing in the below thread and final changes have been > made to the KIP wiki based on these

[jira] [Commented] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

2016-11-29 Thread Harald Kirsch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705770#comment-15705770 ] Harald Kirsch commented on KAFKA-1194: -- Now I tried exactly the SiphonRelease mention

Re: [DISCUSS] 0.10.1.1 Plan

2016-11-29 Thread Ismael Juma
I disagree, but let's discuss it another time and in a separate thread. :) Ismael On Tue, Nov 29, 2016 at 4:30 PM, radai wrote: > designing kafka code for stable extensibility is a worthy and noble cause. > however, seeing as there are no such derivatives out in the wild yet i > think investing

[jira] [Commented] (KAFKA-4443) Controller should send UpdateMetadataRequest prior to LeaderAndIsrRequest during failover

2016-11-29 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705851#comment-15705851 ] Jun Rao commented on KAFKA-4443: [~lindong], thanks for filing the jira. A couple of quest

[jira] [Updated] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3042: --- Labels: reliability (was: ) > updateIsr should stop after failed several times due to zkVersion issue

[jira] [Updated] (KAFKA-4443) Controller should send UpdateMetadataRequest prior to LeaderAndIsrRequest during failover

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4443?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4443: --- Labels: reliability (was: ) > Controller should send UpdateMetadataRequest prior to LeaderAndIsrReque

[jira] [Commented] (KAFKA-4450) Add missing 0.10.1.x upgrade tests and ensure ongoing compatibility checks

2016-11-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4450?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705874#comment-15705874 ] Ewen Cheslack-Postava commented on KAFKA-4450: -- Yeah, that makes sense [~ijum

[GitHub] kafka pull request #2187: Revert "KAFKA-4345; Run decktape test for each pul...

2016-11-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2187 --- 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-4345) Run decktape test for each pull request

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

[jira] [Updated] (KAFKA-4450) Add missing 0.10.1.x upgrade tests and ensure ongoing compatibility checks

2016-11-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4450?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4450: - Fix Version/s: (was: 0.10.1.1) > Add missing 0.10.1.x upgrade tests and ensure

[GitHub] kafka pull request #2172: HOTFIX: KAFKA-4245 follow up: remove gradle wrappe...

2016-11-29 Thread ewencp
Github user ewencp closed the pull request at: https://github.com/apache/kafka/pull/2172 --- 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-4245) BlockingChannel#connect hides all exceptions

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

[jira] [Commented] (KAFKA-4450) Add missing 0.10.1.x upgrade tests and ensure ongoing compatibility checks

2016-11-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4450?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15705948#comment-15705948 ] Ismael Juma commented on KAFKA-4450: Sounds good [~ewencp]. > Add missing 0.10.1.x up

[jira] [Reopened] (KAFKA-4345) Run decktape test for each pull request

2016-11-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava reopened KAFKA-4345: -- > Run decktape test for each pull request > ---

[jira] [Updated] (KAFKA-4345) Run ducktape test for each pull request

2016-11-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4345: - Summary: Run ducktape test for each pull request (was: Run decktape test for each

[GitHub] kafka pull request #1814: MINOR: doc fix related to monitoring consumer lag.

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

[GitHub] kafka pull request #2168: KAFKA-4443; Controller should send UpdateMetadataR...

2016-11-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2168 --- 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-4443) Controller should send UpdateMetadataRequest prior to LeaderAndIsrRequest during failover

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

[jira] [Updated] (KAFKA-4439) Add a builder to NetworkClient

2016-11-29 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4439: --- Status: Patch Available (was: Open) > Add a builder to NetworkClient > --

[jira] [Updated] (KAFKA-4457) Add a command to list the broker version information

2016-11-29 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4457: --- Status: Patch Available (was: Open) > Add a command to list the broker version information >

[jira] [Updated] (KAFKA-4439) Add a builder to NetworkClient

2016-11-29 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4439: --- Affects Version/s: 0.10.0.2 > Add a builder to NetworkClient > --

[jira] [Commented] (KAFKA-1696) Kafka should be able to generate Hadoop delegation tokens

2016-11-29 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706077#comment-15706077 ] Ashish K Singh commented on KAFKA-1696: --- It has been a while, since any progress has

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

2016-11-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706109#comment-15706109 ] Guozhang Wang commented on KAFKA-4039: -- Discussed offline, we will move this out of 0

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

2016-11-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4039: - Fix Version/s: (was: 0.10.1.1) 0.10.2.0 > Exit Strategy: using exceptions i

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-11-29 Thread Becket Qin
+1 (binding) with a minor comment below: The wiki suggests we are going to have a new interface setTombstone() in the ProducerRecord. I am wondering if there is any considerations given to just having the constructor take an additional field so the ProducerRecord remains not modifiable? Thanks,

[GitHub] kafka pull request #2145: MINOR: Remove unused code in `LeaderAndIsr`, `ApiU...

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

[VOTE] KIP-93: Improve invalid timestamp handling in Kafka Streams

2016-11-29 Thread Matthias J. Sax
I’d like to start the voting process for KIP-93: https://cwiki.apache.org/confluence/display/KAFKA/KIP-93%3A+Improve+invalid+timestamp+handling+in+Kafka+Streams -Matthias signature.asc Description: OpenPGP digital signature

[jira] [Issue Comment Deleted] (KAFKA-4271) The console consumer fails on Windows with new consumer is used

2016-11-29 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4271?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-4271: --- Comment: was deleted (was: [~yuyan] Thanks for the pointer, but when I look at the script {{k

[jira] [Commented] (KAFKA-4271) The console consumer fails on Windows with new consumer is used

2016-11-29 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4271?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706138#comment-15706138 ] Vahid Hashemian commented on KAFKA-4271: [~yuyan] Thanks for the pointer, but when

[jira] [Created] (KAFKA-4462) Improved Kafka Client Compatibility Policy

2016-11-29 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-4462: -- Summary: Improved Kafka Client Compatibility Policy Key: KAFKA-4462 URL: https://issues.apache.org/jira/browse/KAFKA-4462 Project: Kafka Issue Type: Impr

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

2016-11-29 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706166#comment-15706166 ] Guozhang Wang commented on KAFKA-4405: -- Thanks [~enothereska]. > Kafka consumer impr

[jira] [Updated] (KAFKA-4462) Improved Kafka Client Compatibility Policy

2016-11-29 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4462?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4462: --- Description: A proposal to improve the compatibility policy of the Kafka client by supporting

[jira] [Resolved] (KAFKA-4448) Add missing licenses to ducktape related files

2016-11-29 Thread Raghav Kumar Gautam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4448?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raghav Kumar Gautam resolved KAFKA-4448. Resolution: Invalid The problematic patch has been reverted, so the issue is no long

[jira] [Commented] (KAFKA-1696) Kafka should be able to generate Hadoop delegation tokens

2016-11-29 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706194#comment-15706194 ] Sriharsha Chintalapani commented on KAFKA-1696: --- [~singhashish] [~omkreddy]

[jira] [Commented] (KAFKA-4447) Controller resigned but it also acts as a controller for a long time

2016-11-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706205#comment-15706205 ] Guozhang Wang commented on KAFKA-4447: -- About the controller re-write: I do not know

[jira] [Commented] (KAFKA-1696) Kafka should be able to generate Hadoop delegation tokens

2016-11-29 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706209#comment-15706209 ] Ashish K Singh commented on KAFKA-1696: --- [~omkreddy] would you be able to provide an

[DISCUSS] KIP-97: Improved Kafka Client RPC Compatibility Policy

2016-11-29 Thread Colin McCabe
Hi all, I've been thinking about a KIP to improve the Kafka client's compatibility policy. If you're interested, please check out: https://cwiki.apache.org/confluence/display/KAFKA/KIP-97%3A+Improved+Kafka+Compatibility+Policy cheers, Colin

Re: [DISCUSS] KIP-97: Improved Kafka Client RPC Compatibility Policy

2016-11-29 Thread Colin McCabe
Sorry, that link should be: https://cwiki.apache.org/confluence/display/KAFKA/KIP-97%3A+Improved+Kafka+Client+RPC+Compatibility+Policy On Tue, Nov 29, 2016, at 11:04, Colin McCabe wrote: > Hi all, > > I've been thinking about a KIP to improve the Kafka client's > compatibility policy. If you'r

[jira] [Commented] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

2016-11-29 Thread Soumyajit Sahu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706222#comment-15706222 ] Soumyajit Sahu commented on KAFKA-1194: --- Thanks for confirming that [~haraldk]. Glad

Re: [VOTE] KIP-93: Improve invalid timestamp handling in Kafka Streams

2016-11-29 Thread Bill Bejeck
+1 (non-binding) Thanks, Bill On Tue, Nov 29, 2016 at 1:34 PM, Matthias J. Sax wrote: > I’d like to start the voting process for KIP-93: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 93%3A+Improve+invalid+timestamp+handling+in+Kafka+Streams > > -Matthias > >

[jira] [Resolved] (KAFKA-4427) Skip topicGroups with no tasks

2016-11-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4427?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4427. -- Resolution: Fixed Issue resolved by pull request 2171 [https://github.com/apache/kafka/pull/2171

[jira] [Created] (KAFKA-4463) Setup travis-ci integration for ducktape tests

2016-11-29 Thread Raghav Kumar Gautam (JIRA)
Raghav Kumar Gautam created KAFKA-4463: -- Summary: Setup travis-ci integration for ducktape tests Key: KAFKA-4463 URL: https://issues.apache.org/jira/browse/KAFKA-4463 Project: Kafka Issu

[jira] [Commented] (KAFKA-4447) Controller resigned but it also acts as a controller for a long time

2016-11-29 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706252#comment-15706252 ] Onur Karaman commented on KAFKA-4447: - Hey [~guozhang]. I've been working on a detaile

[jira] [Created] (KAFKA-4464) Clean shutdown of broker fails due to controller error

2016-11-29 Thread Xavier Lange (JIRA)
Xavier Lange created KAFKA-4464: --- Summary: Clean shutdown of broker fails due to controller error Key: KAFKA-4464 URL: https://issues.apache.org/jira/browse/KAFKA-4464 Project: Kafka Issue Type

[jira] [Updated] (KAFKA-4464) Clean shutdown of broker fails due to controller error

2016-11-29 Thread Xavier Lange (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Lange updated KAFKA-4464: Affects Version/s: 0.10.1.0 > Clean shutdown of broker fails due to controller error > -

[jira] [Updated] (KAFKA-4462) Improved Kafka Client Compatibility Policy

2016-11-29 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4462?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4462: --- Description: A proposal to improve the compatibility policy of the Kafka client by supporting

[jira] [Created] (KAFKA-4465) Create docker image and scripts for running tests locally

2016-11-29 Thread Raghav Kumar Gautam (JIRA)
Raghav Kumar Gautam created KAFKA-4465: -- Summary: Create docker image and scripts for running tests locally Key: KAFKA-4465 URL: https://issues.apache.org/jira/browse/KAFKA-4465 Project: Kafka

[jira] [Commented] (KAFKA-4464) Clean shutdown of broker fails due to controller error

2016-11-29 Thread Xavier Lange (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706283#comment-15706283 ] Xavier Lange commented on KAFKA-4464: - Here is my kafka broker config: {code} kafka@8

[jira] [Created] (KAFKA-4466) Add support to ducktape to run only a part of all tests

2016-11-29 Thread Raghav Kumar Gautam (JIRA)
Raghav Kumar Gautam created KAFKA-4466: -- Summary: Add support to ducktape to run only a part of all tests Key: KAFKA-4466 URL: https://issues.apache.org/jira/browse/KAFKA-4466 Project: Kafka

[jira] [Created] (KAFKA-4467) Run tests on travis-ci using docker

2016-11-29 Thread Raghav Kumar Gautam (JIRA)
Raghav Kumar Gautam created KAFKA-4467: -- Summary: Run tests on travis-ci using docker Key: KAFKA-4467 URL: https://issues.apache.org/jira/browse/KAFKA-4467 Project: Kafka Issue Type: Sub

[GitHub] kafka pull request #2174: MINOR: Make release notes script check resolutions...

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

Re: [DISCUSS] KIP-97: Improved Kafka Client RPC Compatibility Policy

2016-11-29 Thread Ashish Singh
Hello Colin, In the KIP you mentioned that currently the client uses supported api versions information to check if the server supports its desired versions. Not sure, if that is true. I had put together a PR for KAFKA-3600, to do that, but it never went in. Also, I could not find how you plan to

[GitHub] kafka pull request #1028: KAFKA-3123: Follower Broker cannot start if offset...

2016-11-29 Thread soumyajit-sahu
Github user soumyajit-sahu closed the pull request at: https://github.com/apache/kafka/pull/1028 --- 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-3123) Follower Broker cannot start if offsets are already out of range

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

[jira] [Commented] (KAFKA-3123) Follower Broker cannot start if offsets are already out of range

2016-11-29 Thread Soumyajit Sahu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706415#comment-15706415 ] Soumyajit Sahu commented on KAFKA-3123: --- I am closing this previously opened (and no

[GitHub] kafka pull request #2169: KAFKA-4415; Reduce time to create and send Metadat...

2016-11-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2169 --- 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-4415) Reduce time to create and send MetadataUpdateRequest

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

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-11-29 Thread Ignacio Solis
I'm ok with 32 bit keys and leaving the interpretation out of this discussion/KIP. Nacho On Tue, Nov 29, 2016 at 3:35 AM, Michael Pearce wrote: > I assume, that after a period of a week, that there is no concerns now > with points 1, and 2 and now we have agreement that headers are useful and >

[jira] [Commented] (KAFKA-4271) The console consumer fails on Windows with new consumer is used

2016-11-29 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4271?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15706513#comment-15706513 ] Vahid Hashemian commented on KAFKA-4271: [~yuyan] Thanks for the pointer. I can co

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

2016-11-29 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4427: Skip topic groups with no tasks [me] MINOR: Make release notes script check resolutions to avoid spurious [becket.qin] KAFKA-4415; Reduce time to create and send UpdateMetadataRequest

[jira] [Assigned] (KAFKA-4458) Add per partition metrics for in-sync and assigned replica count

2016-11-29 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Léauté reassigned KAFKA-4458: Assignee: Xavier Léauté > Add per partition metrics for in-sync and assigned replica count

Re: [DISCUSS] 0.10.1.1 Plan

2016-11-29 Thread Ignacio Solis
Some of the changes in the 0.10.1 branch already are not bug fixes. Some break compatibility. Having said that, at this level we should maintain a stable API and leave any changes for real version bumps. This should be only a bugfix release. Nacho On Tue, Nov 29, 2016 at 8:35 AM, Ismael Juma

Re: [DISCUSS] 0.10.1.1 Plan

2016-11-29 Thread Ismael Juma
Which changes break compatibility in the 0.10.1 branch? It would be good to fix before the release goes out. Ismael On 29 Nov 2016 9:09 pm, "Ignacio Solis" wrote: > Some of the changes in the 0.10.1 branch already are not bug fixes. Some > break compatibility. > > Having said that, at this leve

  1   2   >