[GitHub] kafka pull request #2774: MINOR: StreamThread should catch InvalidTopicExcep...

2017-03-30 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2774 MINOR: StreamThread should catch InvalidTopicException You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka minor-fix-reset-0102

[jira] [Commented] (KAFKA-4987) Topic creation allows invalid config values on running brokers

2017-03-30 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15950237#comment-15950237 ] huxi commented on KAFKA-4987: - Did you use a higher version of client talking to a lower version of broker? >

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Onur Karaman
It seems like there generally has been the assumption that the broker needs to know about this delay either from its own config or provided over the wire from clients. Is this actually true? One alternative I don't think was mentioned was to make this delay concept be completely client-side.

Build failed in Jenkins: kafka-0.10.2-jdk7 #116

2017-03-30 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4689; Disable system tests for consumer hard failures [wangguoz] MINOR: Increase max.poll time for streams consumers -- [...truncated 257.99

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

2017-03-30 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Dong Lin
+1 (non-binding) Thanks! On Thu, Mar 30, 2017 at 6:03 PM, Becket Qin wrote: > +1 Thanks for the KIP! > > On Thu, Mar 30, 2017 at 12:55 PM, Jason Gustafson > wrote: > > > +1 Thanks for the KIP! > > > > On Thu, Mar 30, 2017 at 12:51 PM, Guozhang Wang

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Becket Qin
+1 Thanks for the KIP! On Thu, Mar 30, 2017 at 12:55 PM, Jason Gustafson wrote: > +1 Thanks for the KIP! > > On Thu, Mar 30, 2017 at 12:51 PM, Guozhang Wang > wrote: > > > +1 > > > > Sorry about the previous email, Gmail seems be collapsing them into a >

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

2017-03-30 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Map `mkString` format updated to default java format [ismael] KAFKA-4902; Utils#delete should correctly handle I/O errors and symlinks [ismael] MINOR: Doc change related to ZK

[GitHub] kafka pull request #2756: KAFKA-4818: Implement transactional producer

2017-03-30 Thread mjsax
Github user mjsax closed the pull request at: https://github.com/apache/kafka/pull/2756 --- 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-4818) Implement transactional producer

2017-03-30 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15950098#comment-15950098 ] ASF GitHub Bot commented on KAFKA-4818: --- Github user mjsax closed the pull request at:

[jira] [Reopened] (KAFKA-4842) Streams integration tests occasionally fail with connection error

2017-03-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4842?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reopened KAFKA-4842: -- Re-open the issue and assigning to [~mjsax] > Streams integration tests occasionally fail with

[jira] [Assigned] (KAFKA-4842) Streams integration tests occasionally fail with connection error

2017-03-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4842?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-4842: Assignee: Matthias J. Sax > Streams integration tests occasionally fail with connection

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

2017-03-30 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Update possible errors in OffsetFetchResponse -- [...truncated 900.84 KB...]

[jira] [Created] (KAFKA-4990) Add API stubs, config parameters, and request types

2017-03-30 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4990: -- Summary: Add API stubs, config parameters, and request types Key: KAFKA-4990 URL: https://issues.apache.org/jira/browse/KAFKA-4990 Project: Kafka Issue

[jira] [Created] (KAFKA-4989) Improve assertions for consumer hard failure system tests to handle possible event loss

2017-03-30 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4989: -- Summary: Improve assertions for consumer hard failure system tests to handle possible event loss Key: KAFKA-4989 URL: https://issues.apache.org/jira/browse/KAFKA-4989

Re: [VOTE] KIP-120: Cleanup Kafka Streams builder API

2017-03-30 Thread Guozhang Wang
+1. On Thu, Mar 30, 2017 at 1:18 AM, Damian Guy wrote: > Thanks Matthias. > > +1 > > On Thu, 23 Mar 2017 at 22:40 Matthias J. Sax > wrote: > > > Hi, > > > > I would like to start the VOTE on KIP-120: > > > >

[GitHub] kafka pull request #2770: MINOR: Increase max.poll time for streams consumer...

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2770 --- 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-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

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

[jira] [Resolved] (KAFKA-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

2017-03-30 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4689?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4689. Resolution: Fixed Fix Version/s: 0.10.2.1 0.11.0.0 Issue resolved by pull

[GitHub] kafka pull request #2771: KAFKA-4689: Disable system tests for consumer hard...

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2771 --- 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] [Updated] (KAFKA-4791) Kafka Streams - unable to add state stores when using wildcard topics on the source

2017-03-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4791?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4791: - Resolution: Fixed Fix Version/s: 0.10.2.1 0.11.0.0 Status:

[jira] [Commented] (KAFKA-4791) Kafka Streams - unable to add state stores when using wildcard topics on the source

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

[GitHub] kafka pull request #2618: KAFKA-4791: unable to add state store with regex m...

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

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

2017-03-30 Thread Dong Lin
Thanks Jun! Hi all, Thanks for all the comments. I am going to open the voting thread if there is no further concern with the KIP. Dong On Thu, Mar 30, 2017 at 3:19 PM, Jun Rao wrote: > Hi, Dong, > > I don't have further concerns. If there are no more comments from other >

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-03-30 Thread Dong Lin
Hi all, Thanks for all the comments. I am going to open the voting thread if there is no further concern with the KIP. Dong On Wed, Mar 15, 2017 at 5:25 PM, Ismael Juma wrote: > Thanks for the updates Dong, they look good to me. > > Ismael > > On Wed, Mar 15, 2017 at 5:50

[jira] [Updated] (KAFKA-4987) Topic creation allows invalid config values on running brokers

2017-03-30 Thread dan norwood (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4987?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] dan norwood updated KAFKA-4987: --- Affects Version/s: (was: 0.10.2.0) > Topic creation allows invalid config values on running

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

2017-03-30 Thread Jun Rao
Hi, Dong, I don't have further concerns. If there are no more comments from other people, we can start the vote. Thanks, Jun On Thu, Mar 30, 2017 at 10:59 AM, Dong Lin wrote: > Hey Jun, > > Thanks much for the comment! Do you think we start vote for KIP-112 and > KIP-113

[jira] [Commented] (KAFKA-4810) SchemaBuilder should be more lax about checking that fields are unset if they are being set to the same value

2017-03-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949951#comment-15949951 ] Ewen Cheslack-Postava commented on KAFKA-4810: -- [~pshk4r] Great! You're already a contributor

[jira] [Commented] (KAFKA-4344) Exception when accessing partition, offset and timestamp in processor class

2017-03-30 Thread saiprasad mishra (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949949#comment-15949949 ] saiprasad mishra commented on KAFKA-4344: - Hi All Lot of people reached out to me on how to use

[jira] [Created] (KAFKA-4988) JVM crash when running on Alpine Linux

2017-03-30 Thread Vincent Rischmann (JIRA)
Vincent Rischmann created KAFKA-4988: Summary: JVM crash when running on Alpine Linux Key: KAFKA-4988 URL: https://issues.apache.org/jira/browse/KAFKA-4988 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #2767: Vagrant provisioning fixes

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2767 --- 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 #2762: MINOR: Ensure streaming iterator is closed by Fetc...

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2762 --- 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] [Updated] (KAFKA-4987) Topic creation allows invalid config values on running brokers

2017-03-30 Thread dan norwood (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4987?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] dan norwood updated KAFKA-4987: --- Affects Version/s: 0.10.0.1 > Topic creation allows invalid config values on running brokers >

[jira] [Updated] (KAFKA-4986) Add producer per task support

2017-03-30 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4986: --- Description: Add new config parameter {{processing_guarantee}} and enable "producer per task"

[jira] [Updated] (KAFKA-4986) Add producer per task support

2017-03-30 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4986: --- Assignee: Matthias J. Sax Status: Patch Available (was: Open)

[jira] [Created] (KAFKA-4987) Topic creation allows invalid config values on running brokers

2017-03-30 Thread dan norwood (JIRA)
dan norwood created KAFKA-4987: -- Summary: Topic creation allows invalid config values on running brokers Key: KAFKA-4987 URL: https://issues.apache.org/jira/browse/KAFKA-4987 Project: Kafka

[jira] [Work started] (KAFKA-4923) Add Exactly-Once Semantics

2017-03-30 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4923?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4923 started by Matthias J. Sax. -- > Add Exactly-Once Semantics > -- > >

[jira] [Created] (KAFKA-4986) Add producer per task support

2017-03-30 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4986: -- Summary: Add producer per task support Key: KAFKA-4986 URL: https://issues.apache.org/jira/browse/KAFKA-4986 Project: Kafka Issue Type: Sub-task

[GitHub] kafka pull request #2773: Add exactly-once config to StreamsConfig

2017-03-30 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2773 Add exactly-once config to StreamsConfig Enable producer per task if exactly-once config is enabled. You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request #2764: MINOR: reduce amount of verbose printing

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

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2017-03-30 Thread Rajini Sivaram
I have made a minor change to the callback handler interface to pass in the JAAS configuration entries in *configure,* to work with the multiple listener configuration introduced in KIP-103. I have also renamed the interface to AuthenticateCallbackHandler instead of AuthCallbackHandler to avoid

[jira] [Commented] (KAFKA-4980) testReprocessingFromScratch unit test failure

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

[GitHub] kafka pull request #2757: KAFKA-4980: testReprocessingFromScratch unit test ...

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2757 --- 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] [Updated] (KAFKA-4980) testReprocessingFromScratch unit test failure

2017-03-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4980: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

Re: [VOTE] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-30 Thread Matthias J. Sax
+1 I am closing this vote now. The KIP got accepted with +3 binding (Jason, Jay, Ram) and +6 non-binding (Eno, Damian, Apruva, Bill, Michael, Matthias) votes. Thanks for voting! -Matthias On 3/29/17 10:27 AM, Michael Noll wrote: > +1 (non-binding) > > On Wed, Mar 29, 2017 at 6:38 PM,

[jira] [Commented] (KAFKA-4980) testReprocessingFromScratch unit test failure

2017-03-30 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949694#comment-15949694 ] Matthias J. Sax commented on KAFKA-4980: PR: https://github.com/apache/kafka/pull/2757 >

[jira] [Updated] (KAFKA-4980) testReprocessingFromScratch unit test failure

2017-03-30 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4980: --- Status: Patch Available (was: Open) > testReprocessingFromScratch unit test failure >

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Jason Gustafson
+1 Thanks for the KIP! On Thu, Mar 30, 2017 at 12:51 PM, Guozhang Wang wrote: > +1 > > Sorry about the previous email, Gmail seems be collapsing them into a > single thread on my inbox. > > Guozhang > > On Thu, Mar 30, 2017 at 11:34 AM, Guozhang Wang >

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Guozhang Wang
+1 Sorry about the previous email, Gmail seems be collapsing them into a single thread on my inbox. Guozhang On Thu, Mar 30, 2017 at 11:34 AM, Guozhang Wang wrote: > Damian, could you create a new thread for the voting process? > > Thanks! > > Guozhang > > On Thu, Mar 30,

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Damian Guy
Hi Guozhang, This was a new thread! Damian On Thu, 30 Mar 2017 at 19:34, Guozhang Wang wrote: > Damian, could you create a new thread for the voting process? > > Thanks! > > Guozhang > > On Thu, Mar 30, 2017 at 10:33 AM, Bill Bejeck wrote: > > >

[jira] [Commented] (KAFKA-4208) Add Record Headers

2017-03-30 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4208?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949654#comment-15949654 ] ASF GitHub Bot commented on KAFKA-4208: --- GitHub user michaelandrepearce opened a pull request:

[GitHub] kafka pull request #2772: KAFKA-4208: Add Record Headers

2017-03-30 Thread michaelandrepearce
GitHub user michaelandrepearce opened a pull request: https://github.com/apache/kafka/pull/2772 KAFKA-4208: Add Record Headers As per KIP-82 Adding record headers api to ProducerRecord, ConsumerRecord Support to convert from protocol to api added Kafka Producer, Kafka

[jira] [Assigned] (KAFKA-4980) testReprocessingFromScratch unit test failure

2017-03-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-4980: Assignee: Matthias J. Sax > testReprocessingFromScratch unit test failure >

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Guozhang Wang
Damian, could you create a new thread for the voting process? Thanks! Guozhang On Thu, Mar 30, 2017 at 10:33 AM, Bill Bejeck wrote: > +1(non-binding) > > On Thu, Mar 30, 2017 at 1:30 PM, Eno Thereska > wrote: > > > +1 (non binding) > > > > Thanks >

[GitHub] kafka pull request #2653: MINOR: Update possible errors in OffsetFetchRespon...

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

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

2017-03-30 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: FetchRequest.Builder maxBytes for version <3 -- [...truncated 1.54 MB...]

[jira] [Commented] (KAFKA-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

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

[GitHub] kafka pull request #2771: KAFKA-4689: Disable system tests for consumer hard...

2017-03-30 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2771 KAFKA-4689: Disable system tests for consumer hard failures See the JIRA for the full details. Essentially the test assertions depend on receiving reliable events from the consumer processes, but

[jira] [Commented] (KAFKA-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

2017-03-30 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949526#comment-15949526 ] Jason Gustafson commented on KAFKA-4689: I think I finally see what is happening here. I looked at

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

2017-03-30 Thread Dong Lin
Hey Jun, Thanks much for the comment! Do you think we start vote for KIP-112 and KIP-113 if there is no further concern? Dong On Thu, Mar 30, 2017 at 10:40 AM, Jun Rao wrote: > Hi, Dong, > > Ok, so it seems that in solution (2), if the tool exits successfully, then > we

[jira] [Commented] (KAFKA-4973) Transient failure of AdminClientTest.testDeleteRecordsWithException

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

[GitHub] kafka pull request #2760: KAFKA-4973; Fix transient failure of AdminClientTe...

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2760 --- 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-4692) Transient test failure in org.apache.kafka.streams.integration.KStreamAggregationDedupIntegrationTest

2017-03-30 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949491#comment-15949491 ] ASF GitHub Bot commented on KAFKA-4692: --- Github user original-brownbear closed the pull request at:

[GitHub] kafka pull request #2761: KAFKA-4692: Make testNo thread safe

2017-03-30 Thread original-brownbear
Github user original-brownbear closed the pull request at: https://github.com/apache/kafka/pull/2761 --- 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

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

2017-03-30 Thread Jun Rao
Hi, Dong, Ok, so it seems that in solution (2), if the tool exits successfully, then we know for sure that all replicas will be in the right log dirs. Solution (1) doesn't guarantee that. That seems better and we can go with your current solution then. Thanks, Jun On Fri, Mar 24, 2017 at 4:28

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Bill Bejeck
+1(non-binding) On Thu, Mar 30, 2017 at 1:30 PM, Eno Thereska wrote: > +1 (non binding) > > Thanks > Eno > > On 30 Mar 2017, at 18:01, Matthias J. Sax wrote: > > > > +1 > > > > On 3/30/17 3:46 AM, Damian Guy wrote: > >> Hi All, > >> > >> I'd like

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Eno Thereska
+1 (non binding) Thanks Eno > On 30 Mar 2017, at 18:01, Matthias J. Sax wrote: > > +1 > > On 3/30/17 3:46 AM, Damian Guy wrote: >> Hi All, >> >> I'd like to start the voting thread on KIP-134: >>

[jira] [Created] (KAFKA-4985) kafka-acls should resolve dns names and accept ip ranges

2017-03-30 Thread Ryan P (JIRA)
Ryan P created KAFKA-4985: - Summary: kafka-acls should resolve dns names and accept ip ranges Key: KAFKA-4985 URL: https://issues.apache.org/jira/browse/KAFKA-4985 Project: Kafka Issue Type:

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Matthias J. Sax
+1 On 3/30/17 3:46 AM, Damian Guy wrote: > Hi All, > > I'd like to start the voting thread on KIP-134: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-134%3A+Delay+initial+consumer+group+rebalance > > Thanks, > Damian > signature.asc Description: OpenPGP digital signature

Re: [DISCUSS] KIP-136: Add Listener name and Security Protocol name to SelectorMetrics tags

2017-03-30 Thread Roger Hoover
Edo, Thanks for the proposal. This looks great to me. Cheers, Roger On Thu, Mar 30, 2017 at 8:51 AM, Edoardo Comar wrote: > Hi all, > > We created KIP-136: Add Listener name and Security Protocol name to > SelectorMetrics tags > >

[jira] [Commented] (KAFKA-4476) Kafka Streams gets stuck if metadata is missing

2017-03-30 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949374#comment-15949374 ] Matthias J. Sax commented on KAFKA-4476: That's a different issues, and we have a fix for this

[DISCUSS] KIP-137: Enhance TopicCommand --describe to show topics marked for deletion

2017-03-30 Thread Mickael Maison
Hi all, We created KIP-137: Enhance TopicCommand --describe to show topics marked for deletion https://cwiki.apache.org/confluence/display/KAFKA/KIP-137%3A+Enhance+TopicCommand+--describe+to+show+topics+marked+for+deletion Please help review the KIP. You feedback is appreciated! Thanks

[GitHub] kafka pull request #2770: MINOR: Increase max.poll time for streams consumer...

2017-03-30 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2770 MINOR: Increase max.poll time for streams consumers You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka

[jira] [Commented] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-03-30 Thread lakshminarayanasyamala (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949356#comment-15949356 ] lakshminarayanasyamala commented on KAFKA-4984: --- Hi Ait haj Slimane, Can you add some more

[jira] [Commented] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-03-30 Thread lakshminarayanasyamala (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949351#comment-15949351 ] lakshminarayanasyamala commented on KAFKA-4984: --- Hi Ait haj Slimane, Can you add some more

Re: [DISCUSS] KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user

2017-03-30 Thread Mayuresh Gharat
Hi Ismael, I have updated the KIP. Let me know if everything looks fine then I will begin voting. Thanks, Mayuresh On Wed, Mar 29, 2017 at 9:06 AM, Mayuresh Gharat wrote: > Hi Ismael, > > I agree. I will change the compatibility para and start voting. > > Thanks,

[DISCUSS] KIP-136: Add Listener name and Security Protocol name to SelectorMetrics tags

2017-03-30 Thread Edoardo Comar
Hi all, We created KIP-136: Add Listener name and Security Protocol name to SelectorMetrics tags https://cwiki.apache.org/confluence/display/KAFKA/KIP-136%3A+Add+Listener+name+and+Security+Protocol+name+to+SelectorMetrics+tags Please help review the KIP. You feedback is appreciated! cheers,

[jira] [Updated] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-03-30 Thread Ait haj Slimane (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ait haj Slimane updated KAFKA-4984: --- Priority: Critical (was: Major) > Unable to produce or consume when enabling authentication

[jira] [Updated] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-03-30 Thread Ait haj Slimane (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ait haj Slimane updated KAFKA-4984: --- Description: I have a problem while trying to produce or consume on kerberos enabled cluster.

[jira] [Created] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-03-30 Thread Ait haj Slimane (JIRA)
Ait haj Slimane created KAFKA-4984: -- Summary: Unable to produce or consume when enabling authentication SASL/Kerberos Key: KAFKA-4984 URL: https://issues.apache.org/jira/browse/KAFKA-4984 Project:

[jira] [Commented] (KAFKA-4981) Add connection-accept-rate and connection-prepare-rate metrics

2017-03-30 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949229#comment-15949229 ] Edoardo Comar commented on KAFKA-4981: -- Hi [~ijuma] for the 3rd time today :-) ... does this need a

[jira] [Commented] (KAFKA-4981) Add connection-accept-rate and connection-prepare-rate metrics

2017-03-30 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949223#comment-15949223 ] Edoardo Comar commented on KAFKA-4981: -- Thanks I know the block is skipped for PLAINTEXT. Actually

[jira] [Commented] (KAFKA-4981) Add connection-accept-rate and connection-prepare-rate metrics

2017-03-30 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949221#comment-15949221 ] Edoardo Comar commented on KAFKA-4981: -- as per comments on the pull request, the 'prepared' metric

[jira] [Updated] (KAFKA-4983) Test failure: kafka.api.ConsumerBounceTest.testSubscribeWhenTopicUnavailable

2017-03-30 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4983?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Magnus Edenhill updated KAFKA-4983: --- Description: The PR builder encountered this test failure:

[jira] [Created] (KAFKA-4983) Test failure: kafka.api.ConsumerBounceTest.testSubscribeWhenTopicUnavailable

2017-03-30 Thread Magnus Edenhill (JIRA)
Magnus Edenhill created KAFKA-4983: -- Summary: Test failure: kafka.api.ConsumerBounceTest.testSubscribeWhenTopicUnavailable Key: KAFKA-4983 URL: https://issues.apache.org/jira/browse/KAFKA-4983

[jira] [Commented] (KAFKA-4476) Kafka Streams gets stuck if metadata is missing

2017-03-30 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949128#comment-15949128 ] Magnus Edenhill commented on KAFKA-4476: Directed here from KAFKA-4482. Happened again on trunk

[jira] [Commented] (KAFKA-4937) Batch resetting offsets in Streams' StoreChangelogReader

2017-03-30 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949081#comment-15949081 ] ASF GitHub Bot commented on KAFKA-4937: --- GitHub user dguy opened a pull request:

[GitHub] kafka pull request #2769: KAFKA-4937: Batch resetting offsets in Streams' St...

2017-03-30 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2769 KAFKA-4937: Batch resetting offsets in Streams' StoreChangelogReader change `consumer.position` so that it always updates any partitions that need an update. Keep track of partitions that

[jira] [Commented] (KAFKA-4982) Add listener tag to socket-server-metrics.connection-... metrics

2017-03-30 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949079#comment-15949079 ] Ismael Juma commented on KAFKA-4982: Yes. > Add listener tag to socket-server-metrics.connection-...

[jira] [Commented] (KAFKA-4982) Add listener tag to socket-server-metrics.connection-... metrics

2017-03-30 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15949036#comment-15949036 ] Edoardo Comar commented on KAFKA-4982: -- [~ijuma] does this need a kip ? > Add listener tag to

[jira] [Assigned] (KAFKA-4982) Add listener tag to socket-server-metrics.connection-... metrics

2017-03-30 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar reassigned KAFKA-4982: Assignee: Edoardo Comar > Add listener tag to socket-server-metrics.connection-... metrics

[jira] [Created] (KAFKA-4982) Add listener tag to socket-server-metrics.connection-... metrics

2017-03-30 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-4982: Summary: Add listener tag to socket-server-metrics.connection-... metrics Key: KAFKA-4982 URL: https://issues.apache.org/jira/browse/KAFKA-4982 Project: Kafka

[GitHub] kafka pull request #2768: KAFKA-4981 Add connection-accept-rate and connecti...

2017-03-30 Thread edoardocomar
GitHub user edoardocomar opened a pull request: https://github.com/apache/kafka/pull/2768 KAFKA-4981 Add connection-accept-rate and connection-prepare-rate KAFKA-4981 Add connection-accept-rate and connection-prepare-rate metrics added metrics per network processor with the

[GitHub] kafka pull request #2767: Vagrant provisioning fixes

2017-03-30 Thread edenhill
GitHub user edenhill opened a pull request: https://github.com/apache/kafka/pull/2767 Vagrant provisioning fixes You can merge this pull request into a Git repository by running: $ git pull https://github.com/edenhill/kafka harden_provision Alternatively you can review and

[GitHub] kafka pull request #2595: MINOR: Fix typos in javadoc and code comments

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2595 --- 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 #2620: MINOR: Doc change related to ZK sasl configs

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2620 --- 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-4902) Utils#delete should correctly handle I/O errors and symlinks

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

[GitHub] kafka pull request #2691: KAFKA-4902: Utils#delete should correctly handle I...

2017-03-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2691 --- 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] [Resolved] (KAFKA-4902) Utils#delete should correctly handle I/O errors and symlinks

2017-03-30 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4902?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4902. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 2691

[GitHub] kafka pull request #2709: MINOR: Map `mkString` format updated to default ja...

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

  1   2   >