Re: ProducerRecord/Consumer MetaData/Headers

2016-09-22 Thread Michael Pearce
Hi again, Sorry to be nudging this, but it seems I'm still unable to create a page in the KIP Proposal area. Just don't want to be forgotten about between all the emails. Cheers Mike From: Michael Pearce Sent: Monday, September 19, 2016 6:19 PM To: dev@k

[jira] [Commented] (KAFKA-4178) Replication Throttling: Consolidate Rate Classes

2016-09-22 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15512485#comment-15512485 ] Ben Stopford commented on KAFKA-4178: - Thanks Joel. Actually this comment, from that t

Unable to locate auto.create.topics.enable=true path for KafkaProducer

2016-09-22 Thread UMESH CHAUDHARY
Hi Mates, I was trying to understand that if auto.create.topics.enable=true then how KafkaProducer first creates the topic and sends messages to it. What I saw: private Future doSend(ProducerRecord record, Callback callback) method in KafkaProducer.java. What I failed to get: When getting meta

Re: ProducerRecord/Consumer MetaData/Headers

2016-09-22 Thread Ismael Juma
Sorry for the delay, you should have access now. Ismael On Thu, Sep 22, 2016 at 8:15 AM, Michael Pearce wrote: > Hi again, > > Sorry to be nudging this, but it seems I'm still unable to create a page > in the KIP Proposal area. Just don't want to be forgotten about between all > the emails. > >

Re: [VOTE] KIP-54: Sticky Partition Assignment Strategy

2016-09-22 Thread Mickael Maison
+1 (non-binding) On Thu, Sep 15, 2016 at 8:32 PM, Bill Bejeck wrote: > +1 > > On Thu, Sep 15, 2016 at 5:16 AM, Rajini Sivaram < > rajinisiva...@googlemail.com> wrote: > >> +1 (non-binding) >> >> On Wed, Sep 14, 2016 at 12:37 AM, Jason Gustafson >> wrote: >> >> > Thanks for the KIP. +1 from me. >

[GitHub] kafka pull request #1899: HOTFIX: Decrease commit interval

2016-09-22 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/1899 HOTFIX: Decrease commit interval The original commit interval of 30 seconds might be too large in some cases, e.g., when the verifier finishes before those 30 seconds have elapsed. You can merg

Does Kafka 0.9 can guarantee not loss data

2016-09-22 Thread Kafka
Hi all, in terms of topic, we create a topic with 6 partition,and each with 3 replicas. in terms of producer,when we send message with ack -1 using sync interface. in terms of brokers,we set min.insync.replicas to 2. after we review the kafka broker’s code,we know that w

[GitHub] kafka pull request #1900: fix ambiguity in docs - consuming by multiple cons...

2016-09-22 Thread pilloPl
GitHub user pilloPl opened a pull request: https://github.com/apache/kafka/pull/1900 fix ambiguity in docs - consuming by multiple consumers, but by exact… In doc it stays "Our topic is divided into a set of totally ordered partitions, each of which is consumed by one con

[jira] [Commented] (KAFKA-4114) Allow for different "auto.offset.reset" strategies for different input streams

2016-09-22 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4114?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15513498#comment-15513498 ] Bill Bejeck commented on KAFKA-4114: Hi Matthias, I've started working on this and

[jira] [Created] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-09-22 Thread Andrew Grasso (JIRA)
Andrew Grasso created KAFKA-4205: Summary: NullPointerException in fetchOffsetsBefore Key: KAFKA-4205 URL: https://issues.apache.org/jira/browse/KAFKA-4205 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-4206) Improve handling of invalid credentials to mitigate DOS issue (especially on SSL listeners)

2016-09-22 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-4206: Summary: Improve handling of invalid credentials to mitigate DOS issue (especially on SSL listeners) Key: KAFKA-4206 URL: https://issues.apache.org/jira/browse/KAFKA-4206

[jira] [Updated] (KAFKA-4206) Improve handling of invalid credentials to mitigate DOS issue (especially on SSL listeners)

2016-09-22 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar updated KAFKA-4206: - Description: The current handling of invalid credentials (ie wrong user/password) is to let the {

[jira] [Updated] (KAFKA-4206) Improve handling of invalid credentials to mitigate DOS issue (especially on SSL listeners)

2016-09-22 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar updated KAFKA-4206: - Description: The current handling of invalid credentials (ie wrong user/password) is to let the {

[GitHub] kafka pull request #1898: KAFKA-3782: Ensure heartbeat thread restarted afte...

2016-09-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1898 --- 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-3782) Transient failure with kafkatest.tests.connect.connect_distributed_test.ConnectDistributedTest.test_bounce.clean=True

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

[jira] [Resolved] (KAFKA-3782) Transient failure with kafkatest.tests.connect.connect_distributed_test.ConnectDistributedTest.test_bounce.clean=True

2016-09-22 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3782?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3782. -- Resolution: Fixed Fix Version/s: (was: 0.10.0.1) 0.10.2.0 Issue re

[jira] [Updated] (KAFKA-4202) Facing error while trying to create the Producer.

2016-09-22 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4202?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4202: - Component/s: (was: KafkaConnect) > Facing error while trying to create the Pro

[jira] [Updated] (KAFKA-4202) Facing error while trying to create the Producer.

2016-09-22 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4202?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4202: - Assignee: (was: Ewen Cheslack-Postava) > Facing error while trying to create t

[jira] [Created] (KAFKA-4207) Partitions stopped after a rapid restart of a broker

2016-09-22 Thread Dustin Cote (JIRA)
Dustin Cote created KAFKA-4207: -- Summary: Partitions stopped after a rapid restart of a broker Key: KAFKA-4207 URL: https://issues.apache.org/jira/browse/KAFKA-4207 Project: Kafka Issue Type: Bu

[GitHub] kafka pull request #1885: Fix comments in KStreamKStreamJoinTest

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

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

2016-09-22 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3782: Ensure heartbeat thread restarted after rebalance woken up -- [...truncated 1019 lines...] kafka.utils.ReplicationUtilsTest > testGetLeaderIsrAndEpochForParti

Re: [DISCUSS] KIP-72 Allow Sizing Incoming Request Queue in Bytes

2016-09-22 Thread radai
As discussed in the KIP call, I have updated the kip-72 page ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-72%3A+Allow+putting+a+bound+on+memory+consumed+by+Incoming+requests) to record both configuration validations and implementation concerns. I've also implemented channel muting/unmuti

Re: Unable to locate auto.create.topics.enable=true path for KafkaProducer

2016-09-22 Thread Thakrar, Jayesh
Umesh, That is a Kafka broker level config/parameter and not part of the producer. Jayesh On 9/22/16, 3:09 AM, "UMESH CHAUDHARY" wrote: Hi Mates, I was trying to understand that if auto.create.topics.enable=true then how KafkaProducer first creates the topic and sends messages to

Jenkins build is back to normal : kafka-0.10.1-jdk7 #4

2016-09-22 Thread Apache Jenkins Server
See

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

2016-09-22 Thread Michael Andre Pearce (IG) (JIRA)
Michael Andre Pearce (IG) created KAFKA-4208: Summary: Add Record Headers Key: KAFKA-4208 URL: https://issues.apache.org/jira/browse/KAFKA-4208 Project: Kafka Issue Type: New Feat

Re: ProducerRecord/Consumer MetaData/Headers

2016-09-22 Thread Michael Pearce
Thanks Ismael. So i have made a start at a draft KIP please see this here. https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers I will start a DISCUSS thread shortly. From: isma...@gmail.com on behalf of Ismael Juma Sent: Thur

[DISCUSS] KIP-82 - Add Record Headers

2016-09-22 Thread Michael Pearce
Hi All, I would like to discuss the following KIP proposal: https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers I have some initial ?drafts of roughly the changes that would be needed. This is no where finalized and look forward to the discussion especially as some

[GitHub] kafka pull request #1899: HOTFIX: Decrease commit interval

2016-09-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1899 --- 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-4202) Facing error while trying to create the Producer.

2016-09-22 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15514380#comment-15514380 ] Ewen Cheslack-Postava commented on KAFKA-4202: -- It looks like you may have mi

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

2016-09-22 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3782: Ensure heartbeat thread restarted after rebalance woken up [wangguoz] MINOR: Fix comments in KStreamKStreamJoinTest -- [...truncated 1737 lines...] kafka.ap

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

2016-09-22 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix comments in KStreamKStreamJoinTest -- [...truncated 5985 lines...] kafka.consumer.PartitionAssignorTest > testRangePartitionAssignor PASSED kafka.network.Sock

[GitHub] kafka pull request #1889: MINOR: Increase `zkConnectionTimeout` and timeout ...

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

[GitHub] kafka pull request #1866: MINOR: Add test cases for delays in consumer rebal...

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

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

2016-09-22 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4178) Replication Throttling: Consolidate Rate Classes

2016-09-22 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15514723#comment-15514723 ] Jun Rao commented on KAFKA-4178: [~benstopford], thanks for the patch. I like your idea of

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

2016-09-22 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: Decrease commit interval -- [...truncated 11683 lines...] org.apache.kafka.clients.MetadataTest > testListenerCanUnregister PASSED org.apache.kafka.clients.Metad

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

2016-09-22 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Increase `zkConnectionTimeout` and timeout in [ismael] MINOR: Add test cases for delays in consumer rebalance listener -- [...truncated 3618 lines...] kafka.log.LogT

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

2016-09-22 Thread Soumyajit Sahu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15515023#comment-15515023 ] Soumyajit Sahu commented on KAFKA-2170: --- [~haraldk], I have updated/fixed the https:

Re: Does Kafka 0.9 can guarantee not loss data

2016-09-22 Thread Kafka
@wangguozhang,could you give me some advices. > 在 2016年9月22日,下午6:56,Kafka 写道: > > Hi all, > in terms of topic, we create a topic with 6 partition,and each with 3 > replicas. >in terms of producer,when we send message with ack -1 using sync > interface. > in terms of

Re: Does Kafka 0.9 can guarantee not loss data

2016-09-22 Thread Becket Qin
In order to satisfy a produce response, there are two conditions: A. The leader's high watermark should be higher than the requiredOffset (max offset in that produce request of that partition) B. The number of in sync replica is greater than min.isr. The ultimate goal here is to make sure at least

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

2016-09-22 Thread Harald Kirsch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15515615#comment-15515615 ] Harald Kirsch commented on KAFKA-2170: -- [~soumyajitsahu] This is great news. I assume