[jira] [Commented] (KAFKA-4570) How to transfer extended fields in producing or consuming requests.

2017-02-22 Thread dongeforever (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4570?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15880002#comment-15880002 ] dongeforever commented on KAFKA-4570: - [~ewendland] it seems that I can use sasl to authenticate the

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-22 Thread Manikumar
+1 (non-binding) On Thu, Feb 23, 2017 at 3:27 AM, Mayuresh Gharat wrote: > Hi Jun, > > Thanks a lot for the comments and reviews. > I agree we should log the username. > What I meant by creating KafkaPrincipal was, after this KIP we would not be > required to create

[GitHub] kafka-site pull request #47: More edits on 0.10.2 web docs after the release

2017-02-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka-site/pull/47 --- 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-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-22 Thread Prasanna Gautam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879885#comment-15879885 ] Prasanna Gautam commented on KAFKA-2729: [~junrao] Thanks for looking into this. Do you mind

Fwd: Producer acks=1, clean broker shutdown and data loss

2017-02-22 Thread Nick Travers
I didn't get any hits on this on users@, so I'm forwarding this to dev@. Any explanation for this would be greatly appreciated! -- Forwarded message -- From: Nick Travers Date: Sat, Feb 18, 2017 at 5:04 PM Subject: Producer acks=1, clean broker shutdown and

[jira] [Commented] (KAFKA-4762) Consumer throwing RecordTooLargeException even when messages are not that large

2017-02-22 Thread Huadong Liu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879868#comment-15879868 ] Huadong Liu commented on KAFKA-4762: [~huxi_2b] You are right. [~neoeahit] confirmed that we did gzip

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-02-22 Thread Jay Kreps
Hey Becket, I get the problem we want to solve with this, but I don't think this is something that makes sense as a user controlled knob that everyone sending data to kafka has to think about. It is basically a bug, right? First, as a technical question is it true that using the uncompressed

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-22 Thread Dong Lin
Hey Rajini, I think it makes a lot of sense to use io_thread_units as metric to quota user's traffic here. LGTM overall. I have some questions regarding sensors. - Can you be more specific in the KIP what sensors will be added? For example, it will be useful to specify the name and attributes of

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-02-22 Thread Dong Lin
Hey Becket, I realized that Apurva has already raised similar questions. I think you answered his question by saying that the request size will not be small. I agree that there will be no impact on throughput if we can reach request size limit with compression estimation disabled. But I am not

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-02-22 Thread Dong Lin
Hey Becket, Thanks for the KIP. I have one question here. Suppose producer's batch.size=100 KB, max.in.flight.requests.per.connection=1. Since each ProduceRequest contains one batch per partition, it means that 100 KB compressed data will be produced per partition per round-trip time as of

[GitHub] kafka-site issue #47: More edits on 0.10.2 web docs after the release

2017-02-22 Thread junrao
Github user junrao commented on the issue: https://github.com/apache/kafka-site/pull/47 @guozhangwang : Thanks for the patch. LGTM --- 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

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-22 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879700#comment-15879700 ] Jun Rao commented on KAFKA-2729: Sorry to hear about the impact to production. Grant mentioned ZK session

[GitHub] kafka-site issue #47: More edits on 0.10.2 web docs after the release

2017-02-22 Thread derrickdoo
Github user derrickdoo commented on the issue: https://github.com/apache/kafka-site/pull/47 👍 LGTM --- 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

[jira] [Commented] (KAFKA-4753) KafkaConsumer susceptible to FetchResponse starvation

2017-02-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879636#comment-15879636 ] Jason Gustafson commented on KAFKA-4753: [~onurkaraman] I agree in general. The user-driven IO

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

2017-02-22 Thread Michael Pearce
Im happy to compromise to keep it mutable but move to an append style api. (as in guava interables concat) class Headers { Headers append(Iterable headers); } I don’t think we’d want prepend, this would give the idea of guaranteed ordering, when in actual fact we don’t provide

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-22 Thread Guozhang Wang
Made a pass over the doc, overall LGTM except a minor comment on the throttling implementation: Stated as "Request processing time throttling will be applied on top if necessary." I thought that it meant the request processing time throttling is applied first, but continue reading I found it

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

2017-02-22 Thread Jason Gustafson
The point about usability is fair. It's also reasonable to expect that common use cases such as appending headers should be done efficiently. Perhaps we could compromise with something like this? class Headers { Headers append(Iterable headers); Headers prepend(Iterable headers); } That

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879561#comment-15879561 ] Onur Karaman commented on KAFKA-1895: - I think it's worth defining the relation between the two

[jira] [Commented] (KAFKA-4753) KafkaConsumer susceptible to FetchResponse starvation

2017-02-22 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879558#comment-15879558 ] Onur Karaman commented on KAFKA-4753: - [~ijuma] I don't think mitigation is the way to go. The two

[jira] [Commented] (KAFKA-4674) Frequent ISR shrinking and expanding and disconnects among brokers

2017-02-22 Thread mjuarez (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879553#comment-15879553 ] mjuarez commented on KAFKA-4674: This is happening to us in one our clusters in prod today, across

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-22 Thread Kane Kim (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879554#comment-15879554 ] Kane Kim commented on KAFKA-2729: - In my opinion it doesn't matter what's causing it (in our case that was

[jira] [Updated] (KAFKA-4674) Frequent ISR shrinking and expanding and disconnects among brokers

2017-02-22 Thread mjuarez (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4674?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] mjuarez updated KAFKA-4674: --- Attachment: kafkabroker.20170221.log.zip > Frequent ISR shrinking and expanding and disconnects among brokers

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879543#comment-15879543 ] Armin Braun commented on KAFKA-1895: {quote} I don't know, supporting two deserializers sounds a

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

2017-02-22 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4786; Wait for heartbeat thread to terminate in consumer close -- [...truncated 1.35 MB...] kafka.security.auth.SimpleAclAuthorizerTest >

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879483#comment-15879483 ] Jason Gustafson commented on KAFKA-1895: I don't know, supporting two deserializers sounds a lot

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

2017-02-22 Thread Dong Lin
Hey Jun, Thanks much for the explanation. I have some questions about 21 but that is less important than 20. 20 would require considerable change to the KIP and probably requires weeks to discuss again. Thus I would like to be very sure that we agree on the problems with the current design as you

[GitHub] kafka pull request #2580: MINOR: Move ProtoUtils methods to ApiKeys

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

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-22 Thread Jun Rao
Hi, Rajini, Thanks for the updated KIP. The latest proposal looks good to me. Jun On Wed, Feb 22, 2017 at 2:19 PM, Rajini Sivaram wrote: > Jun/Roger, > > Thank you for the feedback. > > 1. I have updated the KIP to use absolute units instead of percentage. The >

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879445#comment-15879445 ] Armin Braun commented on KAFKA-1895: {quote} Maybe a simpler way to achieve that would be to have a

[jira] [Comment Edited] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879445#comment-15879445 ] Armin Braun edited comment on KAFKA-1895 at 2/22/17 11:16 PM: -- {quote} Maybe

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Stephane Maarek
https://mvnrepository.com/artifact/org.apache.kafka/kafka_2.11 Am I missing something ? On 23 February 2017 at 9:21:08 am, Gwen Shapira (g...@confluent.io) wrote: I saw them in Maven yesterday? On Wed, Feb 22, 2017 at 2:15 PM, Stephane Maarek wrote: > Awesome

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Stephane Maarek
Awesome thanks a lot! When should we expect the dependencies to be released in Maven? (including 2.12 scala) On 23 February 2017 at 8:27:10 am, Jun Rao (j...@confluent.io) wrote: Thanks for driving the release, Ewen. Jun On Wed, Feb 22, 2017 at 12:33 AM, Ewen Cheslack-Postava

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

2017-02-22 Thread Michael Pearce
I wasn’t referring to the headers needing to be copied, im meaning the fact we’d be forcing a new producer record to be created, with all the contents copied. i.e what will happen is utility method will be created or end up being used, which does this, and returns the new ProducerRecord

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879418#comment-15879418 ] Jason Gustafson commented on KAFKA-1895: It's an interesting idea, but it's also a big change to

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

2017-02-22 Thread Michael Pearce
Lazy init can achieve/avoid that. Re the concat, why don’t we implement that inside the Headers rather than causing everyone to implement this as adding headers in interceptors will be a dominant use case. We want a user friendly API. Having as a user having to code this instead of having the

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread James Cheng
Woohoo! Thanks for running the release, Ewen! -James > On Feb 22, 2017, at 12:33 AM, Ewen Cheslack-Postava wrote: > > The Apache Kafka community is pleased to announce the release for Apache > Kafka 0.10.2.0. This is a feature release which includes the completion > of 15

[jira] [Comment Edited] (KAFKA-4787) KafkaStreams close() is not reentrant

2017-02-22 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4787?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879400#comment-15879400 ] Matthias J. Sax edited comment on KAFKA-4787 at 2/22/17 10:55 PM: -- My

[jira] [Commented] (KAFKA-4787) KafkaStreams close() is not reentrant

2017-02-22 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4787?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879400#comment-15879400 ] Matthias J. Sax commented on KAFKA-4787: My bad. Did read carefully enough... However, it seems to

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Ismael Juma
The index is stale, the files are there: http://central.maven.org/maven2/org/apache/kafka/kafka_2.11/0.10.2.0/ http://central.maven.org/maven2/org/apache/kafka/kafka_2.12/0.10.2.0/ Ismael On Wed, Feb 22, 2017 at 10:26 PM, Stephane Maarek < steph...@simplemachines.com.au> wrote: >

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

2017-02-22 Thread Jason Gustafson
I thought the argument was against creating the extra objects unnecessarily (i.e. if they were not accessed). And note that making the Headers immutable doesn't necessarily mean that they need to be copied: you can do a trick like Guava's Iterables.concat to add additional headers without changing

[jira] [Comment Edited] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879170#comment-15879170 ] Armin Braun edited comment on KAFKA-1895 at 2/22/17 10:30 PM: -- [~hachikuji]

[jira] [Comment Edited] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879170#comment-15879170 ] Armin Braun edited comment on KAFKA-1895 at 2/22/17 10:29 PM: -- [~hachikuji]

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Gwen Shapira
I looked here: https://repository.apache.org/#nexus-search;gav~org.apache.kafka~kafka_2.12kw,versionexpand On Wed, Feb 22, 2017 at 2:26 PM, Stephane Maarek < steph...@simplemachines.com.au> wrote: > https://mvnrepository.com/artifact/org.apache.kafka/kafka_2.11 > > Am I missing something ?

[jira] [Comment Edited] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15874894#comment-15874894 ] Armin Braun edited comment on KAFKA-1895 at 2/22/17 10:26 PM: -- [~jkreps]

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

2017-02-22 Thread Michael Pearce
If the argument for not having a map holding the key, value pairs is due to garbage creation of HashMap entry's, forcing the creation of a whole new producer record to simply add a head, surely is creating a-lot more? From: Jason Gustafson

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Gwen Shapira
I saw them in Maven yesterday? On Wed, Feb 22, 2017 at 2:15 PM, Stephane Maarek wrote: > Awesome thanks a lot! When should we expect the dependencies to be released > in Maven? (including 2.12 scala) > > On 23 February 2017 at 8:27:10 am, Jun Rao

Jenkins build is back to normal : kafka-0.10.2-jdk7 #86

2017-02-22 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-22 Thread Rajini Sivaram
Jun/Roger, Thank you for the feedback. 1. I have updated the KIP to use absolute units instead of percentage. The property is called* io_thread_units* to align with the thread count property *num.io.threads*. When we implement network thread utilization quotas, we can add another property

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-22 Thread Prateek Jaipuria (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879358#comment-15879358 ] Prateek Jaipuria commented on KAFKA-2729: - [~granthenke] We don't see any zookeeper

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

2017-02-22 Thread Jason Gustafson
The current producer interceptor API is this: ProducerRecord onSend(ProducerRecord record); So adding a header means creating a new ProducerRecord with a new header added to the current headers and returning it. Would that not work? -Jason On Wed, Feb 22, 2017 at 1:45 PM, Michael

[jira] [Commented] (KAFKA-4787) KafkaStreams close() is not reentrant

2017-02-22 Thread Steven Schlansker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4787?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879336#comment-15879336 ] Steven Schlansker commented on KAFKA-4787: -- I'm not sure I understand why that should behave

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-22 Thread Mayuresh Gharat
Hi Jun, Thanks a lot for the comments and reviews. I agree we should log the username. What I meant by creating KafkaPrincipal was, after this KIP we would not be required to create KafkaPrincipal and if we want to maintain the old logging, we will have to create it as we do today. I will take

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

2017-02-22 Thread Michael Pearce
So how would you have this work if not mutable where interceptors would add headers? Sent using OWA for iPhone From: Jason Gustafson Sent: Wednesday, February 22, 2017 8:42:27 PM To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-82

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-22 Thread Jun Rao
Hi, Mayuresh, For logging the user name, we could do either way. We just need to make sure the expected user name is logged. Also, currently, we are already creating a KafkaPrincipal on every request. +1 on the latest KIP. Thanks, Jun On Tue, Feb 21, 2017 at 8:05 PM, Mayuresh Gharat

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Jun Rao
Thanks for driving the release, Ewen. Jun On Wed, Feb 22, 2017 at 12:33 AM, Ewen Cheslack-Postava wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 0.10.2.0. This is a feature release which includes the completion > of 15 KIPs, over

[jira] [Commented] (KAFKA-4787) KafkaStreams close() is not reentrant

2017-02-22 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4787?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879270#comment-15879270 ] Matthias J. Sax commented on KAFKA-4787: You should set the handler like this: {noformat}

[jira] [Commented] (KAFKA-4778) OOM on kafka-streams instances with high numbers of unreaped Record classes

2017-02-22 Thread Dave Thomas (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879220#comment-15879220 ] Dave Thomas commented on KAFKA-4778: [~guozhang] We set the following topic configuration values:

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-22 Thread Dave Thomas (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879213#comment-15879213 ] Dave Thomas commented on KAFKA-2729: [~granthenke] We don't see brokers recovering. The message we

[jira] [Resolved] (KAFKA-4786) Transient test failure: ConsumerCoordinatorTest.testHeartbeatThreadClose

2017-02-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4786?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4786. Resolution: Fixed Fix Version/s: 0.10.2.1 0.10.3.0 Issue resolved

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

2017-02-22 Thread Jason Gustafson
I think the point on the mutability of Headers is worth discussing a little more. As far as I can tell, once the ProducerRecord (or ConsumerRecord) is constructed, there should be no need to further change the headers. Is that correct? If so, then why not enforce that that is the case through the

[jira] [Commented] (KAFKA-4786) Transient test failure: ConsumerCoordinatorTest.testHeartbeatThreadClose

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

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879170#comment-15879170 ] Armin Braun commented on KAFKA-1895: [~hachikuji] {quote} How would you propose to expose

[GitHub] kafka pull request #2586: KAFKA-4786: Wait for heartbeat thread to terminate...

2017-02-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2586 --- 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-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-22 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879124#comment-15879124 ] Grant Henke commented on KAFKA-2729: I am curious if everyone on this Jira is actually seeing the

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-22 Thread Prateek Jaipuria (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879076#comment-15879076 ] Prateek Jaipuria commented on KAFKA-2729: - Having the same issue with 0.10.1.0 on a 8 node

[jira] [Commented] (KAFKA-3798) Kafka Consumer 0.10.0.0 killed after rebalancing exception

2017-02-22 Thread Shannon Carey (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879059#comment-15879059 ] Shannon Carey commented on KAFKA-3798: -- Recently had this problem with an 0.8.2.2 consumer as well. I

Current offset for partition out of range; reset offset

2017-02-22 Thread Ghosh, Achintya (Contractor)
Hi All, One of the partitions showing the huge lag(21K) and I see the below error in kafkaserver.out log of one of the kafka nodes. Current offset 43294 for partition [PROD_TASK_TOPIC_120,10] out of range; reset offset to 43293 (kafka.server.ReplicaFetcherThread) What is the quick

[jira] [Comment Edited] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879020#comment-15879020 ] Jason Gustafson edited comment on KAFKA-1895 at 2/22/17 7:27 PM: -

[jira] [Commented] (KAFKA-4787) KafkaStreams close() is not reentrant

2017-02-22 Thread Steven Schlansker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4787?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879019#comment-15879019 ] Steven Schlansker commented on KAFKA-4787: -- A simple solution could simply have further close()

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15879020#comment-15879020 ] Jason Gustafson commented on KAFKA-1895: [~original-brownbear] How would you propose to expose

[jira] [Created] (KAFKA-4787) KafkaStreams close() is not reentrant

2017-02-22 Thread Steven Schlansker (JIRA)
Steven Schlansker created KAFKA-4787: Summary: KafkaStreams close() is not reentrant Key: KAFKA-4787 URL: https://issues.apache.org/jira/browse/KAFKA-4787 Project: Kafka Issue Type:

[GitHub] kafka-site pull request #47: More edits on 0.10.2 web docs after the release

2017-02-22 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka-site/pull/47 More edits on 0.10.2 web docs after the release Ping @derrickdoo @ewencp for reviews. You can merge this pull request into a Git repository by running: $ git pull

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

2017-02-22 Thread Jun Rao
Hi, Dong, Jiangjie, 20. (1) I agree that ideally we'd like to use direct RPC for broker-to-broker communication instead of ZK. However, in the alternative design, the failed log directory path also serves as the persistent state for remembering the offline partitions. This is similar to the

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-22 Thread Roger Hoover
Great to see this KIP and the excellent discussion. To me, Jun's suggestion makes sense. If my application is allocated 1 request handler unit, then it's as if I have a Kafka broker with a single request handler thread dedicated to me. That's the most I can use, at least. That allocation

[jira] [Commented] (KAFKA-1895) Investigate moving deserialization and decompression out of KafkaConsumer

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878775#comment-15878775 ] Armin Braun commented on KAFKA-1895: [~ijuma] maybe you have an opinion on the viability of the above

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Vahid S Hashemian
+1 Thanks Ewen. --Vahid From: "Ewen Cheslack-Postava" To: annou...@apache.org, us...@kafka.apache.org, dev@kafka.apache.org, kafka-clie...@googlegroups.com Date: 02/22/2017 12:33 AM Subject:[ANNOUNCE] Apache Kafka 0.10.2.0 Released The Apache Kafka

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

2017-02-22 Thread Michael Pearce
Hi Ismael On point 1, Sure makes sense will update shortly. On point 2, Setter/getter typical to properties/headers api’s traditionally are map styled interfaces and what I believe is most expected styled thus the Key, Value setter. Also it would mean rather than an interface, we would be

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Becket Qin
Thanks Ewen :) On Wed, Feb 22, 2017 at 5:15 AM, Kenny Gorman wrote: > We are excited about this release! Excellent work! > > Thanks > Kenny Gorman > www.eventador.io > > > On Feb 22, 2017, at 2:33 AM, Ewen Cheslack-Postava > wrote: > > > > The Apache

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-22 Thread Jun Rao
Hi, Rajini, Thanks for the updated KIP. A few more comments. 1. A concern of request_time_percent is that it's not an absolute value. Let's say you give a user a 10% limit. If the admin doubles the number of request handler threads, that user now actually has twice the absolute capacity. This

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-22 Thread Ismael Juma
Great work on the proposal and iterating on it based on community feedback. As Jun (and others) said, it's likely that minor changes will happen as the PR is reviewed and additional testing takes place since this is a significant change. I am +1 (binding) on the proposal without optional keys and

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-22 Thread Jason Gustafson
Thanks Jun, let's go with the simple approach (not using attributes) and revise later if we find a compelling reason. I'll update the docs. -Jason On Tue, Feb 21, 2017 at 6:23 PM, Jun Rao wrote: > It seems that it's simpler and more consistent to avoid optional keys and >

[jira] [Commented] (KAFKA-4786) Transient test failure: ConsumerCoordinatorTest.testHeartbeatThreadClose

2017-02-22 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4786?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878599#comment-15878599 ] ASF GitHub Bot commented on KAFKA-4786: --- GitHub user rajinisivaram opened a pull request:

[GitHub] kafka pull request #2586: KAFKA-4786: Wait for heartbeat thread to terminate...

2017-02-22 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/2586 KAFKA-4786: Wait for heartbeat thread to terminate in consumer close You can merge this pull request into a Git repository by running: $ git pull https://github.com/rajinisivaram/kafka

[jira] [Commented] (KAFKA-4569) Transient failure in org.apache.kafka.clients.consumer.KafkaConsumerTest.testWakeupWithFetchDataAvailable

2017-02-22 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878549#comment-15878549 ] Armin Braun commented on KAFKA-4569: [~ijuma][~hachikuji] it looks like there was an attemp at this

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

2017-02-22 Thread Ismael Juma
Hi all, Great to see the progress that has been achieved on this one. :) A few comments regarding the APIs (I'm still reviewing the message format changes): 1. Nit: `getHeaders` in `ProducerRecord` and `ConsumerRecord` should be named `headers` (we avoid the `get` prefix in Kafka) 2. The

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Kenny Gorman
We are excited about this release! Excellent work! Thanks Kenny Gorman www.eventador.io > On Feb 22, 2017, at 2:33 AM, Ewen Cheslack-Postava wrote: > > The Apache Kafka community is pleased to announce the release for Apache > Kafka 0.10.2.0. This is a feature release which

[jira] [Commented] (KAFKA-1935) Consumer should use a separate socket for Coordinator connection

2017-02-22 Thread Dipen Patel (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1935?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878366#comment-15878366 ] Dipen Patel commented on KAFKA-1935: Thanks for the clarifications. If this isn't a good issue for me

[jira] [Commented] (KAFKA-4569) Transient failure in org.apache.kafka.clients.consumer.KafkaConsumerTest.testWakeupWithFetchDataAvailable

2017-02-22 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878251#comment-15878251 ] Ismael Juma commented on KAFKA-4569: [~hachikuji], any thoughts on how we should fix this now that

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-22 Thread Rajini Sivaram
Jun, Thank you for the review. I have reverted to the original KIP that throttles based on request handler utilization. At the moment, it uses percentage, but I am happy to change to a fraction (out of 1 instead of 100) if required. I have added the examples from this discussion to the KIP. Also

[GitHub] kafka pull request #2585: MINOR: Fix potential integer overflow and String.f...

2017-02-22 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2585 MINOR: Fix potential integer overflow and String.format issue You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka

Re: Kafka Connect / Access to OffsetStorageReader from SourceConnector

2017-02-22 Thread Tianji Li
Hi Florian, Just curious, what 'shared storage' you guys use to keep the files before ingested into Kafka? In our case, we could not figure out such a nice distributed+shared file system that is NOT HDFS alike and runs before Kafka. So we use individual harddisks on connector machines and keep

[jira] [Comment Edited] (KAFKA-3866) KerberosLogin refresh time bug and other improvements

2017-02-22 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878152#comment-15878152 ] Andrew Olson edited comment on KAFKA-3866 at 2/22/17 1:06 PM: -- We saw the

[jira] [Commented] (KAFKA-3866) KerberosLogin refresh time bug and other improvements

2017-02-22 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878152#comment-15878152 ] Andrew Olson commented on KAFKA-3866: - We saw the following error when setting a very low expiration

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

2017-02-22 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-4703) Test with two SASL_SSL listeners with different JAAS contexts

2017-02-22 Thread Balint Molnar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4703?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Balint Molnar updated KAFKA-4703: - Description: [~rsivaram] suggested the following in https://github.com/apache/kafka/pull/2406

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

2017-02-22 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4708; Fix transient failure in -- [...truncated 156.87 KB...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

[GitHub] kafka pull request #2584: MINOR: Fix transient failure of testCannotSendToIn...

2017-02-22 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2584 MINOR: Fix transient failure of testCannotSendToInternalTopic It’s a simple matter of creating the internal topic before trying to send to it. Otherwise, we could get an

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

2017-02-22 Thread Apache Jenkins Server
See Changes: [ismael] HOTFIX: Fix version in __init__.py -- [...truncated 322.80 KB...] kafka.server.KafkaConfigTest > testUncleanLeaderElectionDefault STARTED

[jira] [Updated] (KAFKA-4708) Fix Transient failure in BrokerApiVersionsCommandTest.checkBrokerApiVersionCommandOutput

2017-02-22 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4708?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4708: --- Resolution: Fixed Status: Resolved (was: Patch Available) > Fix Transient failure in >

[jira] [Commented] (KAFKA-4195) support throttling on request rate

2017-02-22 Thread Charly Molter (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4195?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15878008#comment-15878008 ] Charly Molter commented on KAFKA-4195: -- For those looking for the KIP:

  1   2   >