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

2016-08-16 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423839#comment-15423839 ] Joel Koshy commented on KAFKA-4050: --- A stack trace should help further clarify. (This is from a thread

Re: [DISCUSS] Java 8 as a minimum requirement

2016-08-16 Thread Alexis Midon
java7 is end of life. http://www.oracle.com/technetwork/java/eol-135779.html +1 On Tue, Aug 16, 2016 at 6:43 AM Ismael Juma wrote: > Hey Harsha, > > I noticed that you proposed that Storm should drop support for Java 7 in > master: > >

[jira] [Updated] (KAFKA-4052) Allow passing properties file to ProducerPerformance

2016-08-16 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-4052: -- Status: Patch Available (was: Open) > Allow passing properties file to ProducerPerformance >

[jira] [Commented] (KAFKA-4052) Allow passing properties file to ProducerPerformance

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

[GitHub] kafka pull request #1749: KAFKA-4052: Allow passing properties file to Produ...

2016-08-16 Thread SinghAsDev
GitHub user SinghAsDev opened a pull request: https://github.com/apache/kafka/pull/1749 KAFKA-4052: Allow passing properties file to ProducerPerformance You can merge this pull request into a Git repository by running: $ git pull https://github.com/SinghAsDev/kafka KAFKA-4052

[GitHub] kafka pull request #1748: KAFKA-3940 Log should check the return value of di...

2016-08-16 Thread imandhan
GitHub user imandhan opened a pull request: https://github.com/apache/kafka/pull/1748 KAFKA-3940 Log should check the return value of dir.mkdirs() This commit changes all the occurrences of dir.mkdirs() with Files.createDirectory(dir.toPath()) You can merge this pull request into

[jira] [Commented] (KAFKA-3940) Log should check the return value of dir.mkdirs()

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

[jira] [Created] (KAFKA-4052) Allow passing properties file to ProducerPerformance

2016-08-16 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-4052: - Summary: Allow passing properties file to ProducerPerformance Key: KAFKA-4052 URL: https://issues.apache.org/jira/browse/KAFKA-4052 Project: Kafka Issue

Re: Strange behavior when turn the system clock back

2016-08-16 Thread Gabriel Ibarra
Hi, Thanks for answering Ismael. I'm sorry I was absent the last days. Here is the link to the issue: https://issues.apache.org/jira/browse/KAFKA-4051 On Thu, Aug 11, 2016 at 5:11 PM, Ismael Juma wrote: > It's probably worth filing a ticket in JIRA. Please also include a

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

2016-08-16 Thread Gabriel Ibarra (JIRA)
Gabriel Ibarra created KAFKA-4051: - Summary: Strange behavior during rebalance when turning the OS clock back Key: KAFKA-4051 URL: https://issues.apache.org/jira/browse/KAFKA-4051 Project: Kafka

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

2016-08-16 Thread Todd Palino (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423400#comment-15423400 ] Todd Palino commented on KAFKA-4050: It appears to be called every time something needs to be

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

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

[GitHub] kafka pull request #1747: KAFKA-4050: Allow configuration of the PRNG used f...

2016-08-16 Thread toddpalino
GitHub user toddpalino opened a pull request: https://github.com/apache/kafka/pull/1747 KAFKA-4050: Allow configuration of the PRNG used for SSL Add an optional configuration for the SecureRandom PRNG implementation, with the default behavior being the same (use the default

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

2016-08-16 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3769: Create new sensors per-thread in KafkaStreams [me] HOTFIX: Re-inserted SimpleBenchmark output for system tests -- [...truncated 12167 lines...]

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

2016-08-16 Thread Jun Rao
Andrey, Thanks for the KIP. +1 Jun On Tue, Aug 16, 2016 at 1:32 PM, Andrey L. Neporada < anepor...@yandex-team.ru> wrote: > Hi! > > I would like to initiate the voting process for KIP-74: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 74%3A+Add+Fetch+Response+Size+Limit+in+Bytes >

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

2016-08-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423368#comment-15423368 ] Jun Rao commented on KAFKA-4050: [~toddpalino], thanks for reporting this. Do you know how often PRNG is

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

2016-08-16 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423354#comment-15423354 ] Ismael Juma commented on KAFKA-4050: Nice find. :) > Allow configuration of the PRNG used for SSL >

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

2016-08-16 Thread Todd Palino (JIRA)
Todd Palino created KAFKA-4050: -- Summary: Allow configuration of the PRNG used for SSL Key: KAFKA-4050 URL: https://issues.apache.org/jira/browse/KAFKA-4050 Project: Kafka Issue Type:

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

2016-08-16 Thread Maysam Yabandeh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423346#comment-15423346 ] Maysam Yabandeh commented on KAFKA-4039: Certainly. I will submit a patch soon. > Exit Strategy:

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

2016-08-16 Thread Andrey L. Neporada
Hi! I would like to initiate the voting process for KIP-74: https://cwiki.apache.org/confluence/display/KAFKA/KIP-74%3A+Add+Fetch+Response+Size+Limit+in+Bytes Thanks, Andrey.

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

2016-08-16 Thread Apache Jenkins Server
See Changes: [me] HOTFIX: Re-inserted SimpleBenchmark output for system tests -- [...truncated 6821 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

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

2016-08-16 Thread Ismael Juma
Hi Andrey, Can you please start a new thread for the vote? Gmail is showing your vote message in the discuss thread. Ismael On Tue, Aug 16, 2016 at 9:15 PM, Andrey L. Neporada < anepor...@yandex-team.ru> wrote: > Hi! > > I would like to initiate the voting process for KIP-74: >

[jira] [Updated] (KAFKA-4048) Connect does not support RetriableException consistently for sinks

2016-08-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4048: --- Description: We only allow for handling {{RetriableException}} from calls to

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

2016-08-16 Thread Maysam Yabandeh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423311#comment-15423311 ] Maysam Yabandeh commented on KAFKA-4039: Revert is certainly an option but I am not sure if the

[jira] [Updated] (KAFKA-4048) Connect does not support RetriableException consistently for sinks

2016-08-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4048: --- Summary: Connect does not support RetriableException consistently for sinks (was: Connect

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

2016-08-16 Thread Andrey L. Neporada
Hi! I would like to initiate the voting process for KIP-74: https://cwiki.apache.org/confluence/display/KAFKA/KIP-74%3A+Add+Fetch+Response+Size+Limit+in+Bytes Thanks, Andrey.

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

2016-08-16 Thread Alexey Ozeritskiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423298#comment-15423298 ] Alexey Ozeritskiy commented on KAFKA-4039: -- Thanks [~maysamyabandeh]. I think we have to decide

[GitHub] kafka pull request #1746: KAFKA-4049: Fix transient failure in RegexSourceIn...

2016-08-16 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/1746 KAFKA-4049: Fix transient failure in RegexSourceIntegrationTest You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[jira] [Commented] (KAFKA-4049) Transient failure in RegexSourceIntegrationTest.testRegexMatchesTopicsAWhenDeleted

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

[jira] [Updated] (KAFKA-4049) Transient failure in RegexSourceIntegrationTest.testRegexMatchesTopicsAWhenDeleted

2016-08-16 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4049?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4049: - Description: There is an hidden assumption in this test case that the created {{TEST-TOPIC-A}}

[jira] [Updated] (KAFKA-4049) Transient failure in RegexSourceIntegrationTest.testRegexMatchesTopicsAWhenDeleted

2016-08-16 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4049?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4049: - Description: There is a race condition in > Transient failure in >

[jira] [Commented] (KAFKA-3410) Unclean leader election and "Halting because log truncation is not allowed"

2016-08-16 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3410?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423285#comment-15423285 ] Guozhang Wang commented on KAFKA-3410: -- [~wushujames] Could you check if KAFKA-3924 fixed this issue?

[jira] [Created] (KAFKA-4049) Transient failure in RegexSourceIntegrationTest.testRegexMatchesTopicsAWhenDeleted

2016-08-16 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4049: Summary: Transient failure in RegexSourceIntegrationTest.testRegexMatchesTopicsAWhenDeleted Key: KAFKA-4049 URL: https://issues.apache.org/jira/browse/KAFKA-4049

[jira] [Commented] (KAFKA-4042) DistributedHerder thread can die because of connector & task lifecycle exceptions

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

[GitHub] kafka pull request #1745: WIP: KAFKA-4042: prevent DistributedHerder thread ...

2016-08-16 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/kafka/pull/1745 WIP: KAFKA-4042: prevent DistributedHerder thread from dying from connector/task lifecycle exceptions You can merge this pull request into a Git repository by running: $ git pull

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

2016-08-16 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3769: Create new sensors per-thread in KafkaStreams -- [...truncated 3429 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

[jira] [Updated] (KAFKA-4042) DistributedHerder thread can die because of connector & task lifecycle exceptions

2016-08-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4042: --- Component/s: KafkaConnect > DistributedHerder thread can die because of connector & task

[jira] [Created] (KAFKA-4048) Connect does not support RetriableException consistently for sources & sinks

2016-08-16 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4048: -- Summary: Connect does not support RetriableException consistently for sources & sinks Key: KAFKA-4048 URL: https://issues.apache.org/jira/browse/KAFKA-4048

[jira] [Commented] (KAFKA-4047) Return more useful information from ConsumerGroupCommand for consumer groups that are rebalancing or manually assigned

2016-08-16 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423212#comment-15423212 ] Vahid Hashemian commented on KAFKA-4047: Thanks for verifying. > Return more useful information

[GitHub] kafka pull request #1744: HOTFIX: Re-inserted system out

2016-08-16 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1744 --- 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] [Reopened] (KAFKA-4045) Investigate feasibility of hooking into RocksDb's cache

2016-08-16 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4045?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reopened KAFKA-4045: - Reopening since it's worth checking with the RocksDb community on future plans around this. >

[jira] [Resolved] (KAFKA-4047) Return more useful information from ConsumerGroupCommand for consumer groups that are rebalancing or manually assigned

2016-08-16 Thread Sean McKibben (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sean McKibben resolved KAFKA-4047. -- Resolution: Duplicate I think 3853 is pretty close, so I'll mark it as a duplicate. > Return

[GitHub] kafka pull request #1744: HOTFIX: Re-inserted system out

2016-08-16 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/1744 HOTFIX: Re-inserted system out You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka hotfix-ducktape-marker Alternatively you

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

2016-08-16 Thread Andrey L. Neporada
Hi! > On 16 Aug 2016, at 20:28, Jun Rao wrote: > > Hi, Andrey, > > I was thinking of just doing 2 for the new fetch request for backward > compatibility. > > It seems there are no more comments on this thread. So, we can probably > start the voting thread once you update

[jira] [Resolved] (KAFKA-3769) KStream job spending 60% of time writing metrics

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

[jira] [Commented] (KAFKA-3769) KStream job spending 60% of time writing metrics

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

[GitHub] kafka pull request #1530: KAFKA-3769: Create new sensors per-thread in Kafka...

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

ZooKeeper performance issue for storing offset

2016-08-16 Thread Xin Jin
Hi, I'm working on streaming systems in AMPLab at UC Berkeley. This article ( https://cwiki.apache.org/confluence/display/KAFKA/Committing+and+fetching+consumer+offsets+in+Kafka) mentioned the ZooKeeper performance issue when consumers store offsets in ZooKeeper. "In Kafka releases through

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

2016-08-16 Thread Jun Rao
Hi, Andrey, I was thinking of just doing 2 for the new fetch request for backward compatibility. It seems there are no more comments on this thread. So, we can probably start the voting thread once you update the wiki. Also, it seems that KIP-73 depends on this KIP. Do you think you will be

[GitHub] kafka pull request #1735: MINOR: Add application id prefix for copartitionGr...

2016-08-16 Thread guozhangwang
Github user guozhangwang closed the pull request at: https://github.com/apache/kafka/pull/1735 --- 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-4047) Return more useful information from ConsumerGroupCommand for consumer groups that are rebalancing or manually assigned

2016-08-16 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423013#comment-15423013 ] Vahid Hashemian commented on KAFKA-4047: [~graphex] This could be a duplicate of issues reported

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

2016-08-16 Thread Jim Jagielski
I'm following along on the thread so for sure! :) > On Aug 16, 2016, at 12:19 PM, Gwen Shapira wrote: > > Absolutely! > > If you have any concrete suggestions for steps we can take to improve > the process, this will be most awesome. We'd love to learn from your > long

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

2016-08-16 Thread Andrey L. Neporada
Hi, Jun! > On 16 Aug 2016, at 18:52, Jun Rao wrote: > > Hi, Andrey, > > For 2, we actually can know the next message size. In LogSegment.read(), we > first use the offset index to find the file position close to the requested > offset and then scan the log forward to find

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

2016-08-16 Thread Gwen Shapira
Absolutely! If you have any concrete suggestions for steps we can take to improve the process, this will be most awesome. We'd love to learn from your long experience in Apache :) Gwen On Tue, Aug 16, 2016 at 6:59 AM, Jim Jagielski wrote: > By being aware of the potential

[jira] [Created] (KAFKA-4047) Return more useful information from ConsumerGroupCommand for consumer groups that are rebalancing or manually assigned

2016-08-16 Thread Sean McKibben (JIRA)
Sean McKibben created KAFKA-4047: Summary: Return more useful information from ConsumerGroupCommand for consumer groups that are rebalancing or manually assigned Key: KAFKA-4047 URL:

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

2016-08-16 Thread Jun Rao
Hi, Andrey, For 2, we actually can know the next message size. In LogSegment.read(), we first use the offset index to find the file position close to the requested offset and then scan the log forward to find the message whose offset is at or larger than the requested offset. By the time we find

[jira] [Commented] (KAFKA-3478) Finer Stream Flow Control

2016-08-16 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15422907#comment-15422907 ] Bill Bejeck commented on KAFKA-3478: Is this task still available and is it a feature that is still in

[jira] [Created] (KAFKA-4046) Expose Cleaner Offset for Compacted Topics

2016-08-16 Thread Sean McKibben (JIRA)
Sean McKibben created KAFKA-4046: Summary: Expose Cleaner Offset for Compacted Topics Key: KAFKA-4046 URL: https://issues.apache.org/jira/browse/KAFKA-4046 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-4023) Add thread id as prefix in Kafka Streams thread logging

2016-08-16 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4023?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15422890#comment-15422890 ] Bill Bejeck commented on KAFKA-4023: Picking this one up. Just let me know if someone is currently

[jira] [Assigned] (KAFKA-4023) Add thread id as prefix in Kafka Streams thread logging

2016-08-16 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck reassigned KAFKA-4023: -- Assignee: Bill Bejeck > Add thread id as prefix in Kafka Streams thread logging >

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

2016-08-16 Thread William Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15422828#comment-15422828 ] William Yu commented on KAFKA-3937: --- [~ijuma] Took a stab at making the fix. I made it against the 0.9.0

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

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

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

2016-08-16 Thread wiyu
GitHub user wiyu opened a pull request: https://github.com/apache/kafka/pull/1743 KAFKA-3937: Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages @ijuma - Creating this PR against 0.9.0 as this is what we're using in prod. I can modify this for trunk

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

2016-08-16 Thread Jayesh Thakrar
All, If there is no discussion, feedback or objection, is there any concern in going to the next step of voting? Thanks,Jayesh From: Jayesh Thakrar To: "dev@kafka.apache.org" Sent: Saturday, August 13, 2016 11:44 PM Subject: [DISCUSS] KIP-59

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

2016-08-16 Thread Jim Jagielski
By being aware of the potential issues, it's easier to address them at the start, and to create a process which does what it can to "ensure" the problems don't pop up :) > On Aug 16, 2016, at 9:48 AM, Ismael Juma wrote: > > Hi Jim, > > Thanks for your feedback. We value the

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

2016-08-16 Thread Ismael Juma
Hi Jim, Thanks for your feedback. We value the community and we definitely want Kafka to remain a fun and friendly place to participate. Under this proposal, volunteers will still be able to do the work when they can. The benefit is that it is likely to reach users faster since the next release

Re: [DISCUSS] Java 8 as a minimum requirement

2016-08-16 Thread Ismael Juma
Hey Harsha, I noticed that you proposed that Storm should drop support for Java 7 in master: http://markmail.org/message/25do6wd3a6g7cwpe It's useful to know what other Apache projects are doing in this regard, so I'm interested in the timeline being proposed for Storm's transition. I could not

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

2016-08-16 Thread Jim Jagielski
The idea of time-based releases make sense. The issue is when they become the tail wagging the dog. Recall that all developers and contributors are assumed to be doing this because they are personally invested in the project. Their is also the assumption that, as such, they are volunteers and do

Re: Change Apache Kafka properties dynamically without restarting servers using Archaius

2016-08-16 Thread Tom Crayford
inline On Mon, Aug 15, 2016 at 5:49 PM, VIJJU CH wrote: > Hello, > > I have some questions related to properties of Apache Kafka. We have a four > node Kafka Cluster which is on Amazon EMR. Currently, in order to change > any properties we do restart the servers to pick the

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

2016-08-16 Thread Ismael Juma
Hi Andrey, On Tue, Aug 16, 2016 at 12:58 PM, Andrey L. Neporada < anepor...@yandex-team.ru> wrote: > > BTW, what is the target version of this KIP? Currently the inter-broker > protocol version in KIP is set to 0.11.0-IV0. Do we want to target for 0.11 > or maybe somewhat earlier? > I suggest

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

2016-08-16 Thread Andrey L. Neporada
Hi Jun! Thanks for feedback. > On 15 Aug 2016, at 20:04, Jun Rao wrote: > > Hi, Andrey, > > Thanks for the update to the wiki. Just a few more minor comments. > > 1. "If *response_max_bytes* parameter is zero ("no limit"), the request is > processed *exactly* as before."

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

2016-08-16 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-4045: --- Summary: Investigate feasibility of hooking into RocksDb's cache Key: KAFKA-4045 URL: https://issues.apache.org/jira/browse/KAFKA-4045 Project: Kafka Issue

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

2016-08-16 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4045?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-4045. - Resolution: Fixed Unfortunately hooking into RocksDb is not possible at this time. End of

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

2016-08-16 Thread Jaikiran Pai
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 against a non-SSL Kafka broker port. As a result, the logs ended up seeing messages like: 17:53:33,722 WARN

[GitHub] kafka pull request #1742: KAFKA-4015: Add new cleanup.policy, compact_and_de...

2016-08-16 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/1742 KAFKA-4015: Add new cleanup.policy, compact_and_delete Added compact_and_delete cleanup.policy to LogConfig. Updated LogCleaner.CleanerThread to also run deletion for any topics configured with

[jira] [Commented] (KAFKA-4015) Add new cleanup.policy, compact_and_delete

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