[jira] [Created] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-01-11 Thread Yuto Kawamura (JIRA)
Yuto Kawamura created KAFKA-4614: Summary: Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex Key: KAFKA-4614 URL: https://issues.apache.org/jira/browse/KAFKA-4614

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-11 Thread Eno Thereska
Thank you, this KIP is now adopted. The votes were: Binding +1: Neha, Guozhang, Ismael, Sriram Non-binding +1: Bill, Matthias, Damian Thank you Eno & Aarti > On 10 Jan 2017, at 04:52, Neha Narkhede wrote: > > +1. Thanks Eno! > On Mon, Jan 9, 2017 at 1:05 PM Guozhang Wang wrote: > >> +1, than

[jira] [Comment Edited] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-01-11 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15817627#comment-15817627 ] Yuto Kawamura edited comment on KAFKA-4614 at 1/11/17 8:19 AM: -

[jira] [Commented] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-01-11 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15817627#comment-15817627 ] Yuto Kawamura commented on KAFKA-4614: -- I've tried to fix this and now I'm testing pa

[jira] [Created] (KAFKA-4615) AdminClient.send function poll without timeout

2017-01-11 Thread Guillaume Grossetie (JIRA)
Guillaume Grossetie created KAFKA-4615: -- Summary: AdminClient.send function poll without timeout Key: KAFKA-4615 URL: https://issues.apache.org/jira/browse/KAFKA-4615 Project: Kafka Issu

[jira] [Commented] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-01-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15817826#comment-15817826 ] Ismael Juma commented on KAFKA-4614: [~kawamuray], thanks for reporting this. Looking

[jira] [Commented] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-01-11 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15817849#comment-15817849 ] huxi commented on KAFKA-4614: - Great catch for the whole things. As for the unmap things, se

[jira] [Created] (KAFKA-4616) Message log is seen when kafka-producer-perf-test.sh is running and any broker restarted in middle in-between

2017-01-11 Thread sandeep kumar singh (JIRA)
sandeep kumar singh created KAFKA-4616: -- Summary: Message log is seen when kafka-producer-perf-test.sh is running and any broker restarted in middle in-between Key: KAFKA-4616 URL: https://issues.apache.org/

[jira] [Comment Edited] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-01-11 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15817849#comment-15817849 ] huxi edited comment on KAFKA-4614 at 1/11/17 10:00 AM: --- Great catch

[jira] [Commented] (KAFKA-4616) Message log is seen when kafka-producer-perf-test.sh is running and any broker restarted in middle in-between

2017-01-11 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4616?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15817886#comment-15817886 ] huxi commented on KAFKA-4616: - Could you set acks = -1 and retry the producer to see if there

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression

2017-01-11 Thread Dongjin Lee
Ismael, I pushed the benchmark code I used, with some updates (iteration: 20 -> 1000). I also updated the KIP page with the updated benchmark results. Please take a review when you are free. The attached screenshot shows how to run the benchmarker. Thanks, Dongjin On Tue, Jan 10, 2017 at 8:03 PM

[jira] [Created] (KAFKA-4617) gradle-generated core eclipse project has incorrect source folder structure

2017-01-11 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-4617: Summary: gradle-generated core eclipse project has incorrect source folder structure Key: KAFKA-4617 URL: https://issues.apache.org/jira/browse/KAFKA-4617 Project: Ka

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression

2017-01-11 Thread Ismael Juma
Thanks Dongjin. I highly recommend using JMH for the benchmark, the existing one has a few problems that could result in unreliable results. Also, it's a bit surprising that LZ4 is producing smaller output than gzip. Is that right? Ismael On Wed, Jan 11, 2017 at 10:20 AM, Dongjin Lee wrote: > I

[jira] [Commented] (KAFKA-4617) gradle-generated core eclipse project has incorrect source folder structure

2017-01-11 Thread Dhwani Katagade (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4617?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15818024#comment-15818024 ] Dhwani Katagade commented on KAFKA-4617: The gradle eclipse plugin seem to have ot

[jira] [Commented] (KAFKA-4617) gradle-generated core eclipse project has incorrect source folder structure

2017-01-11 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4617?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15818047#comment-15818047 ] Edoardo Comar commented on KAFKA-4617: -- Hi [~dhwanikatagade], thanks I saw that threa

Re: Build problems in Eclipse dev environment setup

2017-01-11 Thread Edoardo Comar
Just my personal preference the different output folders have not been annoying to me. I know that the eclipse compile output is different from the output generated running gradle on the CLI and am happy with that. -- Edoardo Comar IBM MessageHub e

[jira] [Work started] (KAFKA-3714) Allow users greater access to register custom streams metrics

2017-01-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3714?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3714 started by Eno Thereska. --- > Allow users greater access to register custom streams metrics > -

Re: Build problems in Eclipse dev environment setup

2017-01-11 Thread Dhwani Katagade
Hi Edoardo, These kind of inconsistencies do not pose problems in most cases. For instance I was able to get things working in Eclipse by doing a full build on CLI and then importing the gradle eclipse plugin generated .project and .classpath files in eclipse. But as a counter example, the g

Subscribe to mailing list

2017-01-11 Thread Barot, Abhishek
Hi Team, I would like to subscribe for the mailing list ( dev@kafka.apache.org ). I would like to post few queries we've around setting up KAFKA cluster in our production Environment with respect to ideal configuration for 6 node cluster across 3 data centers. Than

Re: Subscribe to mailing list

2017-01-11 Thread Dhwani Katagade
Hi Abhishek, Please follow the instructions here https://kafka.apache.org/contact. For your purpose the users list would be more appropriate I feel. -dhwani On Wednesday 11 January 2017 03:15 PM, Barot, Abhishek wrote: Hi Team, I would like to subscribe for the mailing list ( dev@kafka.apa

[GitHub] kafka pull request #2343: [WIP] Client Compatibility

2017-01-11 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2343 [WIP] Client Compatibility Temporary PR to run tests in Jenkins, this will be merged to #2264 once @cmccabe comes online. You can merge this pull request into a Git repository by running: $ git

[GitHub] kafka pull request #2344: MINOR: add file append to connect-log4j.properties

2017-01-11 Thread berryma4
GitHub user berryma4 opened a pull request: https://github.com/apache/kafka/pull/2344 MINOR: add file append to connect-log4j.properties The default log4j for kafka connect should include a log to file. (It would be nice) You can merge this pull request into a Git repository by

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression

2017-01-11 Thread Dongjin Lee
Hi Ismael, 1. In the case of compression output, yes, lz4 is producing the smaller output than gzip. In fact, my benchmark was inspired by  MessageCompressionTest#testCompressSize unit test and the result is same - 396 bytes for gzip and 387 bytes for lz4. 2. I agree that my (former) approach ca

[jira] [Updated] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-01-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4614: --- Labels: latency performance (was: ) > Long GC pause harming broker performance which is caused by mma

[jira] [Commented] (KAFKA-960) Upgrade Metrics to 3.x

2017-01-11 Thread Andrei Nechaev (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-960?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15818554#comment-15818554 ] Andrei Nechaev commented on KAFKA-960: -- Hello everyone, I created a pull request rela

[jira] [Created] (KAFKA-4618) Enable clients to "re-bootstrap" in the event of a full cluster migration

2017-01-11 Thread Dustin Cote (JIRA)
Dustin Cote created KAFKA-4618: -- Summary: Enable clients to "re-bootstrap" in the event of a full cluster migration Key: KAFKA-4618 URL: https://issues.apache.org/jira/browse/KAFKA-4618 Project: Kafka

Re: [DISCUSS] KIP-81: Max in-flight fetches

2017-01-11 Thread Mickael Maison
Rajini, Why do you think we don't want to do the same for brokers ? It feels like brokers would be affected the same way and could end up delaying group/hearbeat requests. Also given queued.max.requests it seems unlikely that small requests (<<1Kb) being allocated outside of the memory pool would

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

2017-01-11 Thread radai
while HTTP-style (string, string) are the most common and most familiar, there is a very significant impact on msg size, especially given that some payloads are literally a few integers (think stock quotes) and would be dwarfed by an http-like header segment. I think we're ok with not allowing for

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Mayuresh Gharat
Hi Dong, As per "If the message's offset is below low_watermark, then it should have been deleted by log retention policy." ---> I am not sure if I understand this correctly. Do you mean to say that the low_watermark will be updated only when the log retention fires on the broker? Thanks, Mayu

Re: [DISCUSS] KIP-81: Max in-flight fetches

2017-01-11 Thread Rajini Sivaram
Mickael, I had based the comment on KIP-72 description where brokers were muting all client channels once memory pool was empty. Having reviewed the PR today, I think it may be fine to delay muting and allocate small buffers outside of the pool. I would still not want to have a config parameter to

[jira] [Commented] (KAFKA-3601) fail fast when newer client connecting to older server

2017-01-11 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3601?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15818703#comment-15818703 ] Colin P. McCabe commented on KAFKA-3601: [~cpennello_opentable], I believe 0.8, 0.

[jira] [Commented] (KAFKA-3601) fail fast when newer client connecting to older server

2017-01-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3601?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15818711#comment-15818711 ] Ismael Juma commented on KAFKA-3601: There's a bug in 0.9.0.1 that prevents it from di

Re: [DISCUSS] KIP-81: Max in-flight fetches

2017-01-11 Thread Mickael Maison
Ok thanks for the clarification. I agree too, I don't want a new config parameter. From the numbers we gathered (see Edoardo's comment above), it shouldn't be too hard to pick a meaningful value On Wed, Jan 11, 2017 at 3:58 PM, Rajini Sivaram wrote: > Mickael, > > I had based the comment on KIP-7

[GitHub] kafka pull request #2340: MINOR: ConfigDef `parseType` exception message upd...

2017-01-11 Thread Kamal15
Github user Kamal15 closed the pull request at: https://github.com/apache/kafka/pull/2340 --- 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 ena

[jira] [Updated] (KAFKA-4481) Relax Kafka Streams API type constraints

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

[GitHub] kafka pull request #2205: KAFKA-4481: relax streams api type contraints

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

[jira] [Commented] (KAFKA-4481) Relax Kafka Streams API type constraints

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

[GitHub] kafka pull request #2345: MINOR: ConfigDef `parseType` exception message upd...

2017-01-11 Thread Kamal15
GitHub user Kamal15 opened a pull request: https://github.com/apache/kafka/pull/2345 MINOR: ConfigDef `parseType` exception message updated. - When Kafka configurations are provided programmatically, for the below configuration incorrect error message gets printed. * co

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

2017-01-11 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: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[GitHub] kafka pull request #1884: KAFKA-4060: Remove zk client dependency in kafka s...

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

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

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

[jira] [Commented] (KAFKA-4577) NPE in ControllerChannelManager.scala::addUpdateMetadataRequestForBrokers

2017-01-11 Thread Scott Reynolds (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15818922#comment-15818922 ] Scott Reynolds commented on KAFKA-4577: --- Nothing interesting was happening, obviousl

[GitHub] kafka pull request #2292: MINOR: Update rocksDB dependency to 5.0.1

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

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-11 Thread Ben Stopford
So I believe we can mark this as Accepted. I've updated the KIP page. Thanks for the input everyone. On Fri, Jan 6, 2017 at 9:31 AM Ben Stopford wrote: > Thanks Joel. I'll fix up the pics to make them consistent on nomenclature. > > > B > > On Fri, Jan 6, 2017 at 2:39 AM Joel Koshy wrote: > > (

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-11 Thread Ben Stopford
Sorry - my mistake. Looks like I still need one more binding vote. Is there a committer out there that could add their vote? B On Wed, Jan 11, 2017 at 6:44 PM Ben Stopford wrote: > So I believe we can mark this as Accepted. I've updated the KIP page. > Thanks for the input everyone. > > On Fri,

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

2017-01-11 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4481: relax streams api type contraints -- [...truncated 18260 lines...] org.apache.kafka.streams.KafkaStreamsTest > testCannotStartOnceClosed PASSED org.apache.k

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-11 Thread Ben Stopford
OK - my mistake was mistaken! There is consensus. This KIP has been accepted. On Wed, Jan 11, 2017 at 6:48 PM Ben Stopford wrote: > Sorry - my mistake. Looks like I still need one more binding vote. Is > there a committer out there that could add their vote? > > B > > On Wed, Jan 11, 2017 at 6:4

[VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Ben Stopford
Looks like there was a good consensus on the discuss thread for KIP-106 so lets move to a vote. Please chime in if you would like to change the default for unclean.leader.election.enabled from true to false. https://cwiki.apache.org/confluence/display/KAFKA/%5BWIP%5D+KIP-106+-+Change+Default+uncl

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

2017-01-11 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #2343: [WIP] Client Compatibility

2017-01-11 Thread ijuma
Github user ijuma closed the pull request at: https://github.com/apache/kafka/pull/2343 --- 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 enabl

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Jason Gustafson
+1 On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > Looks like there was a good consensus on the discuss thread for KIP-106 so > lets move to a vote. > > Please chime in if you would like to change the default for > unclean.leader.election.enabled from true to false. > > https://cwiki.apa

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Onur Karaman
+1 On Wed, Jan 11, 2017 at 11:06 AM, Jason Gustafson wrote: > +1 > > On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > > > Looks like there was a good consensus on the discuss thread for KIP-106 > so > > lets move to a vote. > > > > Please chime in if you would like to change the default

[GitHub] kafka pull request #2182: ConfigDef experimentation - support List and Ma...

2017-01-11 Thread shikhar
Github user shikhar closed the pull request at: https://github.com/apache/kafka/pull/2182 --- 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 ena

[GitHub] kafka pull request #2264: Kafka 4507: The client should send older versions ...

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

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Jay Kreps
+1 On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > Looks like there was a good consensus on the discuss thread for KIP-106 so > lets move to a vote. > > Please chime in if you would like to change the default for > unclean.leader.election.enabled from true to false. > > https://cwiki.apa

[GitHub] kafka pull request #1446: KAFKA-3715: add granular metrics per node

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

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Ian Wrigley
+1 (non-binding) > On Jan 11, 2017, at 11:33 AM, Jay Kreps wrote: > > +1 > > On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > >> Looks like there was a good consensus on the discuss thread for KIP-106 so >> lets move to a vote. >> >> Please chime in if you would like to change the def

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Gwen Shapira
+1 On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > Looks like there was a good consensus on the discuss thread for KIP-106 so > lets move to a vote. > > Please chime in if you would like to change the default for > unclean.leader.election.enabled from true to false. > > https://cwiki.apac

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Apurva Mehta
+1 (non-binding) On Wed, Jan 11, 2017 at 11:45 AM, Gwen Shapira wrote: > +1 > > On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > > Looks like there was a good consensus on the discuss thread for KIP-106 > so > > lets move to a vote. > > > > Please chime in if you would like to change the

[ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Gwen Shapira
The PMC for Apache Kafka has invited Grant Henke to join as a committer and we are pleased to announce that he has accepted! Grant contributed 88 patches, 90 code reviews, countless great comments on discussions, a much-needed cleanup to our protocol and the on-going and critical work on the Admin

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Dong Lin
Hi Mayuresh, low_watermark will be updated when log retention fires on the broker. It may also be updated on the follower when follower receives FetchResponse from leader; and it may be updated on the leader when leader receives PurgeRequest from admin client. Thanks, Dong On Wed, Jan 11, 2017 a

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Sriram Subramanian
Congratulations Grant! On Wed, Jan 11, 2017 at 11:51 AM, Gwen Shapira wrote: > The PMC for Apache Kafka has invited Grant Henke to join as a > committer and we are pleased to announce that he has accepted! > > Grant contributed 88 patches, 90 code reviews, countless great > comments on discussio

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Vahid S Hashemian
Congrats Grant! --Vahid From: Sriram Subramanian To: us...@kafka.apache.org Cc: dev@kafka.apache.org, priv...@kafka.apache.org Date: 01/11/2017 11:58 AM Subject:Re: [ANNOUNCE] New committer: Grant Henke Congratulations Grant! On Wed, Jan 11, 2017 at 11:51 AM, Gwen Shapi

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Jason Gustafson
Congrats! On Wed, Jan 11, 2017 at 11:57 AM, Sriram Subramanian wrote: > Congratulations Grant! > > On Wed, Jan 11, 2017 at 11:51 AM, Gwen Shapira wrote: > > > The PMC for Apache Kafka has invited Grant Henke to join as a > > committer and we are pleased to announce that he has accepted! > > > >

[jira] [Resolved] (KAFKA-3715) Higher granularity streams metrics

2017-01-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3715?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3715. -- Resolution: Fixed > Higher granularity streams metrics > --- >

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Ismael Juma
Congratulations Grant, well deserved. :) Ismael On 11 Jan 2017 7:51 pm, "Gwen Shapira" wrote: > The PMC for Apache Kafka has invited Grant Henke to join as a > committer and we are pleased to announce that he has accepted! > > Grant contributed 88 patches, 90 code reviews, countless great > com

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Ben Stopford
Congrats Grant!! On Wed, 11 Jan 2017 at 20:01, Ismael Juma wrote: > Congratulations Grant, well deserved. :) > > Ismael > > On 11 Jan 2017 7:51 pm, "Gwen Shapira" wrote: > > > The PMC for Apache Kafka has invited Grant Henke to join as a > > committer and we are pleased to announce that he has a

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Ben Stopford
Thanks all. We can consider this accepted. B On Wed, 11 Jan 2017 at 19:49, Apurva Mehta wrote: > +1 (non-binding) > > On Wed, Jan 11, 2017 at 11:45 AM, Gwen Shapira wrote: > > > +1 > > > > On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > > > Looks like there was a good consensus on the

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Jeff Widman
+1 nonbinding. We were bit by this in a production environment. On Wed, Jan 11, 2017 at 11:42 AM, Ian Wrigley wrote: > +1 (non-binding) > > > On Jan 11, 2017, at 11:33 AM, Jay Kreps wrote: > > > > +1 > > > > On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > > > >> Looks like there was a

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Eno Thereska
Congrats! Eno > On 11 Jan 2017, at 20:06, Ben Stopford wrote: > > Congrats Grant!! > On Wed, 11 Jan 2017 at 20:01, Ismael Juma wrote: > >> Congratulations Grant, well deserved. :) >> >> Ismael >> >> On 11 Jan 2017 7:51 pm, "Gwen Shapira" wrote: >> >>> The PMC for Apache Kafka has invited G

[jira] [Commented] (KAFKA-4611) Support custom authentication mechanism

2017-01-11 Thread mahendiran chandrasekar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4611?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15819067#comment-15819067 ] mahendiran chandrasekar commented on KAFKA-4611: I think changes in 0.10.2

[jira] [Resolved] (KAFKA-4611) Support custom authentication mechanism

2017-01-11 Thread mahendiran chandrasekar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4611?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mahendiran chandrasekar resolved KAFKA-4611. Resolution: Workaround Fix Version/s: 0.10.2.0 > Support custom authe

[jira] [Created] (KAFKA-4619) Dissallow to output records with unknown keys in TransformValues

2017-01-11 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4619: -- Summary: Dissallow to output records with unknown keys in TransformValues Key: KAFKA-4619 URL: https://issues.apache.org/jira/browse/KAFKA-4619 Project: Kafka

[jira] [Work started] (KAFKA-4619) Dissallow to output records with unknown keys in TransformValues

2017-01-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4619?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4619 started by Matthias J. Sax. -- > Dissallow to output records with unknown keys in TransformValues >

[jira] [Commented] (KAFKA-4612) Simple Kafka Streams app causes "ClassCastException: java.lang.String cannot be cast to [B"

2017-01-11 Thread Kurt Ostfeld (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4612?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15819134#comment-15819134 ] Kurt Ostfeld commented on KAFKA-4612: - Thank you! You explained the issue, it makes pe

[VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Dong Lin
Hi all, It seems that there is no further concern with the KIP-107. At this point we would like to start the voting process. The KIP can be found at https://cwiki.apache.org/confluence/display/KAFKA/KIP-107%3A+Add+purgeDataBefore%28%29+API+in+AdminClient . Thanks, Dong

[VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Dong Lin
Hi all, It seems that there is no further concern with the KIP-107. At this point we would like to start the voting process. The KIP can be found at https://cwiki.apache.org/confluence/display/KAFKA/KIP- 107%3A+Add+purgeDataBefore%28%29+API+in+AdminClient. Thanks, Dong

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

2017-01-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4064?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4064: --- Assignee: Xavier Léauté (was: Roger Hoover) > Add support for infinite endpoints for range qu

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

2017-01-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4064?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4064: --- Labels: needs-kip (was: ) > Add support for infinite endpoints for range queries in Kafka Str

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

2017-01-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4064?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4064: --- Affects Version/s: 0.10.2.0 > Add support for infinite endpoints for range queries in Kafka St

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

2017-01-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4064?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4064: --- Fix Version/s: (was: 0.10.2.0) 0.10.3.0 > Add support for infinite endp

[VOTE] KIP-107 Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Dong Lin
Hi all, It seems that there is no further concern with the KIP-107. At this point we would like to start the voting process. The KIP can be found at https://cwiki.apache.org/confluence/display/KAFKA/KIP-107 %3A+Add+purgeDataBefore%28%29+API+in+AdminClient. Thanks, Dong

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-11 Thread Dong Lin
Sorry for the duplicated email. It seems that gmail will put the voting email in this thread if I simply replace DISCUSS with VOTE in the subject. On Wed, Jan 11, 2017 at 12:57 PM, Dong Lin wrote: > Hi all, > > It seems that there is no further concern with the KIP-107. At this point > we would

[jira] [Commented] (KAFKA-4612) Simple Kafka Streams app causes "ClassCastException: java.lang.String cannot be cast to [B"

2017-01-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4612?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15819174#comment-15819174 ] Matthias J. Sax commented on KAFKA-4612: If you add {{.through()}} there will be n

[GitHub] kafka pull request #2346: KAFKA-4619: Dissallow to output records with unkno...

2017-01-11 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2346 KAFKA-4619: Dissallow to output records with unknown keys in TransformValues You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka kafka-461

[jira] [Commented] (KAFKA-4619) Dissallow to output records with unknown keys in TransformValues

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

[jira] [Commented] (KAFKA-4612) Simple Kafka Streams app causes "ClassCastException: java.lang.String cannot be cast to [B"

2017-01-11 Thread Kurt Ostfeld (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4612?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15819196#comment-15819196 ] Kurt Ostfeld commented on KAFKA-4612: - Thank you so much! That all makes perfect sense

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

2017-01-11 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-4619) Dissallow to output records with unknown keys in TransformValues

2017-01-11 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4619?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4619: --- Status: Patch Available (was: In Progress) > Dissallow to output records with unknown keys in

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Jay Kreps
Congrats Grant! -Jay On Wed, Jan 11, 2017 at 11:51 AM, Gwen Shapira wrote: > The PMC for Apache Kafka has invited Grant Henke to join as a > committer and we are pleased to announce that he has accepted! > > Grant contributed 88 patches, 90 code reviews, countless great > comments on discussion

[jira] [Commented] (KAFKA-2610) Metrics for SSL handshake

2017-01-11 Thread Felix A Mercado (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2610?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15819238#comment-15819238 ] Felix A Mercado commented on KAFKA-2610: So, i guess I will take a stab at this:

[jira] [Comment Edited] (KAFKA-2610) Metrics for SSL handshake

2017-01-11 Thread Felix A Mercado (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2610?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15819238#comment-15819238 ] Felix A Mercado edited comment on KAFKA-2610 at 1/11/17 9:31 PM: ---

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression

2017-01-11 Thread Ewen Cheslack-Postava
FYI, there's an outstanding patch for getting some JMH benchmarking setup: https://github.com/apache/kafka/pull/1712 I haven't found time to review it (and don't really know JMH well anyway) but it might be worth getting that landed so we can use it for this as well. -Ewen On Wed, Jan 11, 2017 at

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression

2017-01-11 Thread Ismael Juma
That's a good point Ewen. Dongjin, you could use the branch that Ewen linked for the performance testing. It would also help validate the PR. Ismael On Wed, Jan 11, 2017 at 9:38 PM, Ewen Cheslack-Postava wrote: > FYI, there's an outstanding patch for getting some JMH benchmarking setup: > https

[jira] [Updated] (KAFKA-1894) Avoid long or infinite blocking in the consumer

2017-01-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1894?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1894: --- Fix Version/s: (was: 0.10.2.0) 0.10.3.0 > Avoid long or infinite blocking in th

[jira] [Updated] (KAFKA-4507) The client should send older versions of requests to the broker if necessary

2017-01-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4507?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4507: --- Resolution: Fixed Status: Resolved (was: Patch Available) > The client should send older vers

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

2017-01-11 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4507; Clients should support older brokers (KIP-97) [wangguoz] KAFKA-3715: Add granular metrics to Kafka Streams and add hierarhical -- [...truncated 3936 lines...]

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread Becket Qin
Congrats Grant! On Wed, Jan 11, 2017 at 2:17 PM, Kaufman Ng wrote: > Congrats Grant! > > On Wed, Jan 11, 2017 at 4:28 PM, Jay Kreps wrote: > > > Congrats Grant! > > > > -Jay > > > > On Wed, Jan 11, 2017 at 11:51 AM, Gwen Shapira > wrote: > > > > > The PMC for Apache Kafka has invited Grant Hen

Re: [ANNOUNCE] New committer: Grant Henke

2017-01-11 Thread James Cheng
Congrats, Grant!! -James > On Jan 11, 2017, at 11:51 AM, Gwen Shapira wrote: > > The PMC for Apache Kafka has invited Grant Henke to join as a > committer and we are pleased to announce that he has accepted! > > Grant contributed 88 patches, 90 code reviews, countless great > comments on discu

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Guozhang Wang
+1 On Wed, Jan 11, 2017 at 12:09 PM, Jeff Widman wrote: > +1 nonbinding. We were bit by this in a production environment. > > On Wed, Jan 11, 2017 at 11:42 AM, Ian Wrigley wrote: > > > +1 (non-binding) > > > > > On Jan 11, 2017, at 11:33 AM, Jay Kreps wrote: > > > > > > +1 > > > > > > On Wed,

  1   2   >