Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-17 Thread Joel Koshy
On Wed, Aug 17, 2016 at 9:13 PM, Ben Stopford wrote: > > Let's us know if you have any further thoughts on KIP-73, else we'll kick > off a vote. > I think the mechanism for throttling replicas looks good. Just had a few more thoughts on the configuration section. What you

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-17 Thread Joel Koshy
Congrats! On Wed, Aug 17, 2016 at 9:28 PM, Sriram Subramanian wrote: > Congratulations Gwen! > > On Wed, Aug 17, 2016 at 9:24 PM, Neha Narkhede wrote: > > > Congratulations and welcome, Gwen! > > On Wed, Aug 17, 2016 at 6:44 PM Jun Rao

[jira] [Commented] (KAFKA-4050) Allow configuration of the PRNG used for SSL

2016-08-17 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425906#comment-15425906 ] Joel Koshy commented on KAFKA-4050: --- [~toddpalino] I had left a comment about this on the PR - one

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-17 Thread Sriram Subramanian
Congratulations Gwen! On Wed, Aug 17, 2016 at 9:24 PM, Neha Narkhede wrote: > Congratulations and welcome, Gwen! > On Wed, Aug 17, 2016 at 6:44 PM Jun Rao wrote: > > > Hi, Everyone, > > > > Gwen Shapira has been active in the Kafka community since she

Re: [ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-17 Thread Neha Narkhede
Congratulations and welcome, Gwen! On Wed, Aug 17, 2016 at 6:44 PM 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: [DISCUSS] KIP-73: Replication Quotas

2016-08-17 Thread Ben Stopford
Hi everyone Let's us know if you have any further thoughts on KIP-73, else we'll kick off a vote. Thanks B On Friday, 12 August 2016, Jun Rao wrote: > Mayuresh, > > I was thinking of the following. > > If P1 has data and P2 is throttled, we will return empty data for P2

Re: [DISCUSS] KIP-59 - Proposal for a kafka broker command - kafka-brokers.sh

2016-08-17 Thread Gwen Shapira
Jayesh, if you have a github repo with your tool, we can add it to the ecosystem page in the wiki, so more people will find it. On Aug 17, 2016 8:00 PM, "Jayesh Thakrar" wrote: > Thank you Grant and Gwen for the valuable feedback and suggestions.Yes, it > makes

Re: [DISCUSS] KIP-59 - Proposal for a kafka broker command - kafka-brokers.sh

2016-08-17 Thread Jayesh Thakrar
Thank you Grant and Gwen for the valuable feedback and suggestions.Yes, it makes sense to wait for the admin client API.In the meantime, I can spin this off as a standalone tool and allow things to bake, giving the utility more field usage and maturity. As for the endpoint info, I included it

Re: [DISCUSS] KIP-59 - Proposal for a kafka broker command - kafka-brokers.sh

2016-08-17 Thread Grant Henke
Hi Jayesh, Like Gwen said KIP-4 is adding fields and requests to the wire protocols in order to allow all admin tools to talk directly to Kafka and a client api to support those requests. Talking to Kafka as opposed to Zookeeper allows us to leverage authorization, lock down zookeeper, and

[ANNOUNCE] New Kafka PMC member Gwen Shapira

2016-08-17 Thread Jun Rao
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. Congratulations, Gwen! Jun

[jira] [Commented] (KAFKA-4053) Refactor TopicCommand to remove redundant if/else statements

2016-08-17 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4053?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425777#comment-15425777 ] ASF GitHub Bot commented on KAFKA-4053: --- GitHub user sh-z reopened a pull request:

[jira] [Commented] (KAFKA-4053) Refactor TopicCommand to remove redundant if/else statements

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

[GitHub] kafka pull request #1751: KAFKA-4053: remove redundant if/else statements in...

2016-08-17 Thread sh-z
GitHub user sh-z reopened a pull request: https://github.com/apache/kafka/pull/1751 KAFKA-4053: remove redundant if/else statements in TopicCommand[WiP] You can merge this pull request into a Git repository by running: $ git pull https://github.com/sh-z/kafka KAFKA-4053

[GitHub] kafka pull request #1751: KAFKA-4053: remove redundant if/else statements in...

2016-08-17 Thread sh-z
Github user sh-z closed the pull request at: https://github.com/apache/kafka/pull/1751 --- 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 #1751: KAFKA-4053: remove redundant if/else statements in...

2016-08-17 Thread sh-z
Github user sh-z closed the pull request at: https://github.com/apache/kafka/pull/1751 --- 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-4053) Refactor TopicCommand to remove redundant if/else statements

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

[GitHub] kafka pull request #1751: KAFKA-4053: remove redundant if/else statements in...

2016-08-17 Thread sh-z
GitHub user sh-z reopened a pull request: https://github.com/apache/kafka/pull/1751 KAFKA-4053: remove redundant if/else statements in TopicCommand[WiP] You can merge this pull request into a Git repository by running: $ git pull https://github.com/sh-z/kafka KAFKA-4053

[jira] [Commented] (KAFKA-4053) Refactor TopicCommand to remove redundant if/else statements

2016-08-17 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4053?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425773#comment-15425773 ] ASF GitHub Bot commented on KAFKA-4053: --- GitHub user sh-z reopened a pull request:

[jira] [Commented] (KAFKA-4050) Allow configuration of the PRNG used for SSL

2016-08-17 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425756#comment-15425756 ] Todd Palino commented on KAFKA-4050: So first off, yes, the thread dump (which [~jjkoshy] posted)

Re: [DISCUSS] KIP-59 - Proposal for a kafka broker command - kafka-brokers.sh

2016-08-17 Thread Gwen Shapira
Thanks Jayesh. I think this can be a useful addition to Apache Kafka. One potential issue is that you are getting all the information for ZooKeeper. We already added a protocol that allows adding the information to Kafka itself and are in the process of adding an admin client (i.e. Java client,

[jira] [Commented] (KAFKA-4050) Allow configuration of the PRNG used for SSL

2016-08-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425587#comment-15425587 ] Ismael Juma commented on KAFKA-4050: It's clear that we should make this configurable. One question is

[jira] [Updated] (KAFKA-4060) Remove ZkClient dependency in Kafka Streams

2016-08-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4060?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4060: - Assignee: Hojjat Jafarpour > Remove ZkClient dependency in Kafka Streams >

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

2016-08-17 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3888: send consumer heartbeats from a background thread (KIP-62) -- [...truncated 12002 lines...]

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

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

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

2016-08-17 Thread wiyu
Github user wiyu closed the pull request at: https://github.com/apache/kafka/pull/1743 --- 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-4060) Remove ZkClient dependency in Kafka Streams

2016-08-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4060?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4060: - Description: In Kafka Streams we need to dynamically create or update those internal topics

[jira] [Created] (KAFKA-4060) Remove ZkClient dependency in Kafka Streams

2016-08-17 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4060: Summary: Remove ZkClient dependency in Kafka Streams Key: KAFKA-4060 URL: https://issues.apache.org/jira/browse/KAFKA-4060 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-4059) Documentation still refers to AsyncProducer and SyncProducer

2016-08-17 Thread Andrew B (JIRA)
Andrew B created KAFKA-4059: --- Summary: Documentation still refers to AsyncProducer and SyncProducer Key: KAFKA-4059 URL: https://issues.apache.org/jira/browse/KAFKA-4059 Project: Kafka Issue

[jira] [Commented] (KAFKA-1530) howto update continuously

2016-08-17 Thread Alexey Ozeritskiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425454#comment-15425454 ] Alexey Ozeritskiy commented on KAFKA-1530: -- I think this ticket may be closed

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

2016-08-17 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3888: send consumer heartbeats from a background thread (KIP-62) -- [...truncated 5136 lines...] kafka.api.PlaintextProducerSendTest > testWrongSerializer STARTED

Re: [DISCUSS] Time-based releases for Apache Kafka

2016-08-17 Thread Vahid S Hashemian
Time-based releases is a good idea and something that has proved to be working in a number of open source projects. One successful example is Node.js, that goes through two major releases a year. The interesting fact about the two releases is that only one (the even-number release) comes with

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

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

[GitHub] kafka pull request #1757: KAFKA-2170: Fixes for Windows

2016-08-17 Thread soumyajit-sahu
GitHub user soumyajit-sahu opened a pull request: https://github.com/apache/kafka/pull/1757 KAFKA-2170: Fixes for Windows This PR is to revive the older PR https://github.com/apache/kafka/pull/154 made by @mpoindexter (Mike Poindexter). I have replicated his work over the new code

[jira] [Commented] (KAFKA-4058) Failure in org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterReset

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

[GitHub] kafka pull request #1756: KAFKA-4058: Failure in org.apache.kafka.streams.in...

2016-08-17 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/1756 KAFKA-4058: Failure in org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterReset - added retry on ResetTool error You can merge this pull request into a Git

[GitHub] kafka pull request #1732: MINOR: Clarification in producer config documentat...

2016-08-17 Thread vahidhashemian
GitHub user vahidhashemian reopened a pull request: https://github.com/apache/kafka/pull/1732 MINOR: Clarification in producer config documentation You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka

[GitHub] kafka pull request #1732: MINOR: Clarification in producer config documentat...

2016-08-17 Thread vahidhashemian
Github user vahidhashemian 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

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

2016-08-17 Thread Jason Gustafson
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 is hosting some of the assigned partitions. Unless you were planning to modify this behavior, this

[jira] [Commented] (KAFKA-3888) Allow consumer to send heartbeats in background thread (KIP-62)

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

[jira] [Resolved] (KAFKA-3888) Allow consumer to send heartbeats in background thread (KIP-62)

2016-08-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3888?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3888. -- Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1627

[GitHub] kafka pull request #1627: KAFKA-3888: send consumer heartbeats from a backgr...

2016-08-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1627 --- 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] [Created] (KAFKA-4058) Failure in org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterReset

2016-08-17 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4058: Summary: Failure in org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterReset Key: KAFKA-4058 URL:

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

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

[GitHub] kafka pull request #1755: KAFKA-4039: delay invocation of System.exit via Fa...

2016-08-17 Thread maysamyabandeh
GitHub user maysamyabandeh opened a pull request: https://github.com/apache/kafka/pull/1755 KAFKA-4039: delay invocation of System.exit via FatalExitException @resetius would be great if you can confirm that the deadlock no longer manifests with the path. Thanks You can merge

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

2016-08-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4044; log actual socket send/receive buffer size after connecting -- [...truncated 12171 lines...] org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreTest

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

2016-08-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4044; log actual socket send/receive buffer size after connecting -- [...truncated 12204 lines...]

[jira] [Commented] (KAFKA-4050) Allow configuration of the PRNG used for SSL

2016-08-17 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424982#comment-15424982 ] Mayuresh Gharat commented on KAFKA-4050: Just a heads up, this has also been seen as an issue in

Re: kafka broker is dropping the messages after acknowledging librdkafka

2016-08-17 Thread Mazhar Shaikh
Hi Jun, I'm using default configuration (ack=1), changing it t0 all or 2 will not help, as the producer queue will be exhausted is any kafka broker goes down for long time. Thanks. Regards, Mazhar Shaikh. On Wed, Aug 17, 2016 at 8:11 PM, Jun Rao wrote: > Are you using

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

2016-08-17 Thread Oleg Gorobets (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424715#comment-15424715 ] Oleg Gorobets commented on KAFKA-3172: -- I experience the same problem with new consumer API

[jira] [Commented] (KAFKA-4044) log actual socket send/receive buffer size after connecting in Selector

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

[GitHub] kafka pull request #1750: KAFKA-4044: log actual socket send/receive buffer ...

2016-08-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1750 --- 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-4044) log actual socket send/receive buffer size after connecting in Selector

2016-08-17 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4044?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-4044. Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1750

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

2016-08-17 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison reassigned KAFKA-4056: - Assignee: Mickael Maison > Kafka logs values of sensitive configs like passwords >

[jira] [Commented] (KAFKA-4057) Allow to specify the request version and replica ID in kafka.javaapi.FetchRequest

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

[GitHub] kafka pull request #1754: KAFKA-4057: Allow to specify the request version a...

2016-08-17 Thread mimaison
GitHub user mimaison opened a pull request: https://github.com/apache/kafka/pull/1754 KAFKA-4057: Allow to specify the request version and replica ID in ka… …fka.javaapi.FetchRequest - Added new arguments for versionId and replicaId in the constructor instead of using

Re: kafka broker is dropping the messages after acknowledging librdkafka

2016-08-17 Thread Jun Rao
Are you using acks=1 or acks=all in the producer? Only the latter guarantees acked messages won't be lost after leader failure. Thanks, Jun On Wed, Aug 10, 2016 at 11:41 PM, Mazhar Shaikh wrote: > Hi Kafka Team, > > I'm using kafka (kafka_2.11-0.9.0.1) with

[jira] [Commented] (KAFKA-2544) Replication tools wiki page needs to be updated

2016-08-17 Thread Lauren Moos (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424617#comment-15424617 ] Lauren Moos commented on KAFKA-2544: I can take this if someone can add me to the contributors list

[jira] [Created] (KAFKA-4057) Allow to specify the request version and replica ID in kafka.javaapi.FetchRequest

2016-08-17 Thread Mickael Maison (JIRA)
Mickael Maison created KAFKA-4057: - Summary: Allow to specify the request version and replica ID in kafka.javaapi.FetchRequest Key: KAFKA-4057 URL: https://issues.apache.org/jira/browse/KAFKA-4057

[jira] [Updated] (KAFKA-3492) support quota based on authenticated user name

2016-08-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3492?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-3492: -- Fix Version/s: 0.10.1.0 Status: Patch Available (was: Open) > support quota based

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

2016-08-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4019?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reassigned KAFKA-4019: - Assignee: Rajini Sivaram > LogCleaner should grow read/write buffer to max message size

Re: Do not log value of configs that Kafka doesn't recognize

2016-08-17 Thread Jaikiran Pai
Created https://issues.apache.org/jira/browse/KAFKA-4056. -Jaikiran On Wednesday 17 August 2016 06:28 PM, Ismael Juma wrote: Yes, please file a JIRA. Thanks, Ismael On Wed, Aug 17, 2016 at 1:46 PM, Jaikiran Pai wrote: Thanks for the inputs. I think it's fine if

[jira] [Resolved] (KAFKA-4054) Quota related metrics and sensors are never deleted

2016-08-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-4054. --- Resolution: Not A Problem As pointed out by [~ijuma] in KAFKA-3980, metrics and sensors are

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

2016-08-17 Thread jaikiran pai (JIRA)
jaikiran pai created KAFKA-4056: --- Summary: Kafka logs values of sensitive configs like passwords Key: KAFKA-4056 URL: https://issues.apache.org/jira/browse/KAFKA-4056 Project: Kafka Issue

[jira] [Commented] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-08-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424492#comment-15424492 ] Rajini Sivaram commented on KAFKA-3980: --- [~ijuma] Thank you, I will close KAFKA-4054 since the

Re: [DISCUSS] Time-based releases for Apache Kafka

2016-08-17 Thread Mickael Maison
+1 Having better predictability when features will land is a huge benefit. On Tue, Aug 16, 2016 at 5:34 PM, Jim Jagielski wrote: > I'm following along on the thread so for sure! :) > >> On Aug 16, 2016, at 12:19 PM, Gwen Shapira wrote: >> >> Absolutely! >>

Re: Do not log value of configs that Kafka doesn't recognize

2016-08-17 Thread Ismael Juma
Yes, please file a JIRA. Thanks, Ismael On Wed, Aug 17, 2016 at 1:46 PM, Jaikiran Pai wrote: > Thanks for the inputs. > > I think it's fine if Kafka selectively logs values for certain configs and > masks others (the passwords). So from what I understand, this looks

Re: Do not log value of configs that Kafka doesn't recognize

2016-08-17 Thread Jaikiran Pai
Thanks for the inputs. I think it's fine if Kafka selectively logs values for certain configs and masks others (the passwords). So from what I understand, this looks more like a bug where it unintentionally ended up logging these values. Do you want me to file a JIRA for this? -Jaikiran

[jira] [Commented] (KAFKA-3492) support quota based on authenticated user name

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

[GitHub] kafka pull request #1753: KAFKA-3492: Secure quotas for authenticated users

2016-08-17 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/1753 KAFKA-3492: Secure quotas for authenticated users Implementation and tests for secure quotas at and levels as described in KIP-55. Also adds dynamic default quotas for ,

[jira] [Commented] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-08-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424436#comment-15424436 ] Ismael Juma commented on KAFKA-3980: [~rsivaram], did you see my comment above? We have code to expire

Re: Do not log value of configs that Kafka doesn't recognize

2016-08-17 Thread Ismael Juma
Note that we already mask password related config values[1]. The issue is that it only happens after parsing, and the output of `logUnused` is based on the `originals` map, which has the values before parsing: public void logUnused() { for (String key : unused()) log.warn("The

[jira] [Commented] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-08-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424402#comment-15424402 ] Rajini Sivaram commented on KAFKA-3980: --- [~omkreddy] Thanks for the link. I was searching for an

Re: Do not log value of configs that Kafka doesn't recognize

2016-08-17 Thread Manikumar Reddy
During server/client startup, we are logging all the supplied configs. May be we can just mask the password related config values for both valid/invalid configs. On Wed, Aug 17, 2016 at 5:14 PM, Jaikiran Pai wrote: > Any opinion about this proposed change? > >

[jira] [Commented] (KAFKA-3492) support quota based on authenticated user name

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

[GitHub] kafka pull request #1256: KAFKA-3492: Secure quotas for authenticated users

2016-08-17 Thread rajinisivaram
Github user rajinisivaram closed the pull request at: https://github.com/apache/kafka/pull/1256 --- 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

Re: Do not log value of configs that Kafka doesn't recognize

2016-08-17 Thread Jaikiran Pai
Any opinion about this proposed change? -Jaikiran On Tuesday 16 August 2016 02:28 PM, Jaikiran Pai wrote: We are using 0.9.0.1 of Kafka (Java) libraries for our Kafka consumers and producers. In one of our consumers, our consumer config had a SSL specific property which ended up being used

[jira] [Comment Edited] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-08-17 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424315#comment-15424315 ] Manikumar Reddy edited comment on KAFKA-3980 at 8/17/16 11:22 AM: -- as

[jira] [Commented] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-08-17 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424315#comment-15424315 ] Manikumar Reddy commented on KAFKA-3980: as suspected, this may be related to quota leak :

[jira] [Created] (KAFKA-4055) Add system tests for secure quotas

2016-08-17 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-4055: - Summary: Add system tests for secure quotas Key: KAFKA-4055 URL: https://issues.apache.org/jira/browse/KAFKA-4055 Project: Kafka Issue Type: Test

[jira] [Created] (KAFKA-4054) Quota related metrics and sensors are never deleted

2016-08-17 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-4054: - Summary: Quota related metrics and sensors are never deleted Key: KAFKA-4054 URL: https://issues.apache.org/jira/browse/KAFKA-4054 Project: Kafka Issue

Re: kafka broker is dropping the messages after acknowledging librdkafka

2016-08-17 Thread Mazhar Shaikh
Hi Tom, Thank you for responding and sorry for delay. I'm running with all the default configuration provided by kafka. I don't have this config elements in my server.properties file. However the default values specified in kafka documentation are as below (

[jira] [Commented] (KAFKA-3776) Unify store and downstream caching in streams

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

[GitHub] kafka pull request #1752: KAFKA-3776: Unify store and downstream caching in ...

2016-08-17 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/1752 KAFKA-3776: Unify store and downstream caching in streams [WiP] Work-in-progress PoC, not to be merged. You can merge this pull request into a Git repository by running: $ git pull

[jira] [Commented] (KAFKA-4053) Refactor TopicCommand to remove redundant if/else statements

2016-08-17 Thread Umesh Chaudhary (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4053?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424224#comment-15424224 ] Umesh Chaudhary commented on KAFKA-4053: I will do that. Please assign it to me. > Refactor

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

2016-08-17 Thread Alexey Ozeritskiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424201#comment-15424201 ] Alexey Ozeritskiy commented on KAFKA-4039: -- Thanks [~maysamyabandeh] I'll be happy to test your

[GitHub] kafka pull request #1751: remove redundant if/else statements in TopicComman...

2016-08-17 Thread sh-z
GitHub user sh-z opened a pull request: https://github.com/apache/kafka/pull/1751 remove redundant if/else statements in TopicCommand You can merge this pull request into a Git repository by running: $ git pull https://github.com/sh-z/kafka KAFKA-4053 Alternatively you can

[jira] [Created] (KAFKA-4053) Refactor TopicCommand to remove redundant if/else statements

2016-08-17 Thread Shuai Zhang (JIRA)
Shuai Zhang created KAFKA-4053: -- Summary: Refactor TopicCommand to remove redundant if/else statements Key: KAFKA-4053 URL: https://issues.apache.org/jira/browse/KAFKA-4053 Project: Kafka

[jira] [Commented] (KAFKA-4044) log actual socket send/receive buffer size after connecting in Selector

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

[GitHub] kafka pull request #1750: KAFKA-4044: log actual socket send/receive buffer ...

2016-08-17 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/1750 KAFKA-4044: log actual socket send/receive buffer size after connecting in Selector You can merge this pull request into a Git repository by running: $ git pull

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

2016-08-17 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424099#comment-15424099 ] Damian Guy commented on KAFKA-4045: --- https://github.com/facebook/rocksdb/issues/1282 > Investigate