Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-02-02 Thread Guozhang Wang
Hi Dong, Could you elaborate a bit more how controller could affect leaders to switch between all and quorum? Guozhang On Fri, Feb 2, 2018 at 10:12 PM, Dong Lin wrote: > Hey Guazhang, > > Got it. Thanks for the detailed explanation. I guess my point is that we > can probably achieve the best

[jira] [Created] (KAFKA-6530) Use actual first offset of messages when rolling log segment for magic v2

2018-02-02 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6530: -- Summary: Use actual first offset of messages when rolling log segment for magic v2 Key: KAFKA-6530 URL: https://issues.apache.org/jira/browse/KAFKA-6530 Project:

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-02-02 Thread Dong Lin
Hey Guazhang, Got it. Thanks for the detailed explanation. I guess my point is that we can probably achieve the best of both worlds, i.e. maintain the existing behavior of ack="all" while improving the tail latency. Thanks, Dong On Fri, Feb 2, 2018 at 8:43 PM, Guozhang Wang wrote: > Hi Dong,

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-02-02 Thread Guozhang Wang
Hi Dong, Yes, in terms of fault tolerance "quorum" does not do better than "all", as I said, with {min.isr} to X+1 Kafka is able to tolerate X failures only. So if A and B are partitioned off at the same time, then there are two concurrent failures and we do not guarantee all acked messages will b

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

2018-02-02 Thread Apache Jenkins Server
See

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

2018-02-02 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6494; ConfigCommand update to use AdminClient for broker configs -- [...truncated 415.99 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

Build failed in Jenkins: kafka-1.1-jdk7 #7

2018-02-02 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6494; ConfigCommand update to use AdminClient for broker configs -- [...truncated 411.76 KB...] kafka.utils.json.JsonValueTest > testDecodeString ST

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-02-02 Thread Dong Lin
Hey Guozhang, According to the new proposal, with 3 replicas, min.isr=2 and acks="quorum", it seems that acknowledged messages can still be truncated in the network partition scenario you mentioned, right? So I guess the goal is for some user to achieve better tail latency at the cost of potential

Jenkins build is back to normal : kafka-1.1-jdk7 #6

2018-02-02 Thread Apache Jenkins Server
See

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

2018-02-02 Thread Apache Jenkins Server
See Changes: [mjsax] KAFKA-6354: Update KStream JavaDoc using new State Store API (#4456) -- [...truncated 413.27 KB...] kafka.security.auth.SimpleAclAuthorizerTest > tes

Jenkins build is back to normal : kafka-1.0-jdk7 #144

2018-02-02 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-6494) Extend ConfigCommand to update broker config using new AdminClient

2018-02-02 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6494?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6494. Resolution: Fixed Fix Version/s: (was: 1.2.0) 1.1.0 > Extend C

Kafka streams API for .net

2018-02-02 Thread Bykov, Serge
Hello. We are a .net shop and was wondering if you are planning streams API support for .net? If not, where can I find documentation on the REST API around KSQL mentioned in this ticket: https://github.com/confluentinc/confluent-kafka-dotnet/issues/344 Finally, if none of the above, then how c

[jira] [Created] (KAFKA-6529) Broker leaks memory and file descriptors after sudden client disconnects

2018-02-02 Thread Graham Campbell (JIRA)
Graham Campbell created KAFKA-6529: -- Summary: Broker leaks memory and file descriptors after sudden client disconnects Key: KAFKA-6529 URL: https://issues.apache.org/jira/browse/KAFKA-6529 Project: K

Re: [VOTE] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2018-02-02 Thread Matthias J. Sax
Feature freeze for 1.1 passed already, thus, KIP-222 will not be part of 1.1 release. I updated the JIRA with target version 1.2. -Matthias On 2/1/18 3:57 PM, Jeff Widman wrote: > Don't forget to update the wiki page now that the vote has passed--it > currently says this KIP is "under discussion

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-02-02 Thread Guozhang Wang
Hello Litao, Just double checking on the leader election details, do you have time to complete the proposal on that part? Also Jun mentioned one caveat related to KIP-250 on the KIP-232 discussion thread that Dong is working on, I figured it is worth pointing out here with a tentative solution:

[jira] [Created] (KAFKA-6528) Transient failure in DynamicBrokerReconfigurationTest.testThreadPoolResize

2018-02-02 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6528: -- Summary: Transient failure in DynamicBrokerReconfigurationTest.testThreadPoolResize Key: KAFKA-6528 URL: https://issues.apache.org/jira/browse/KAFKA-6528 Project:

[jira] [Created] (KAFKA-6527) Transient failure in DynamicBrokerReconfigurationTest.testDefaultTopicConfig

2018-02-02 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6527: -- Summary: Transient failure in DynamicBrokerReconfigurationTest.testDefaultTopicConfig Key: KAFKA-6527 URL: https://issues.apache.org/jira/browse/KAFKA-6527 Projec

[jira] [Created] (KAFKA-6526) Update controller to handle changes to unclean.leader.election.enable

2018-02-02 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-6526: - Summary: Update controller to handle changes to unclean.leader.election.enable Key: KAFKA-6526 URL: https://issues.apache.org/jira/browse/KAFKA-6526 Project: Kafka

[jira] [Resolved] (KAFKA-6496) NAT and Kafka

2018-02-02 Thread Ronald van de Kuil (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6496?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ronald van de Kuil resolved KAFKA-6496. --- Resolution: Not A Bug > NAT and Kafka > - > > Key: KAFKA-6

[jira] [Resolved] (KAFKA-2925) NullPointerException if FileStreamSinkTask is stopped before initialization finishes

2018-02-02 Thread Robert Yokota (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2925?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Yokota resolved KAFKA-2925. -- Resolution: Cannot Reproduce I wasn't able to reproduce the NPE and by reviewing the code it doe

Re: [VOTE] KIP-206: Add support for UUID serialization and deserialization

2018-02-02 Thread Colin McCabe
Hi Brandon, I think people are generally busy working on the upcoming release now. Sorry for the inconvenience. best, On Fri, Feb 2, 2018, at 07:33, Brandon Kirchner wrote: > I'd really like this to get 2 more binding votes. If that doesn't happen, > how / can this still move forward? Not sur

[jira] [Created] (KAFKA-6525) Connect should allow pluggable encryption for records

2018-02-02 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6525: Summary: Connect should allow pluggable encryption for records Key: KAFKA-6525 URL: https://issues.apache.org/jira/browse/KAFKA-6525 Project: Kafka Issue Typ

Re: [DISCUSS] KIP-252: Extend ACLs to allow filtering based on ip ranges and subnets

2018-02-02 Thread Sönke Liebau
Hi Manikumar, you are right, 5713 is a bit ambiguous about which fields are considered in scope, but I agree that wildcards for Ips are not necessary when we have ranges. I am wondering though, if we might want to extend the scope of this KIP a bit while we are changing acl and authorizer classes

Re: [VOTE] KIP-206: Add support for UUID serialization and deserialization

2018-02-02 Thread Brandon Kirchner
I'd really like this to get 2 more binding votes. If that doesn't happen, how / can this still move forward? Not sure what the procedure is... Brandon K. On Tue, Jan 30, 2018 at 9:56 AM, Mickael Maison wrote: > +1 (non binding) > Thanks for the KIP > > On Tue, Jan 30, 2018 at 9:49 AM, Manikumar

Re: Kafka Log deletion Problem

2018-02-02 Thread Manikumar
looks like log segment is not rotated. You can send more data or adjust log.roll.ms/log.segment.bytes configs to rotate segments. On Fri, Feb 2, 2018 at 7:08 PM, SenthilKumar K wrote: > Hello Experts , We have a Kafka Setup running for our analytics pipeline > ...Below is the broker config ..

Kafka Log deletion Problem

2018-02-02 Thread SenthilKumar K
Hello Experts , We have a Kafka Setup running for our analytics pipeline ...Below is the broker config .. max.message.bytes = 67108864 replica.fetch.max.bytes = 67108864 zookeeper.session.timeout.ms = 7000 replica.socket.timeout.ms = 3 offsets.commit.timeout.ms = 5000 request.timeout.ms = 4000

[jira] [Created] (KAFKA-6524) kafka mirror can't producer internal topic

2018-02-02 Thread Ahmed Madkour (JIRA)
Ahmed Madkour created KAFKA-6524: Summary: kafka mirror can't producer internal topic Key: KAFKA-6524 URL: https://issues.apache.org/jira/browse/KAFKA-6524 Project: Kafka Issue Type: Bug

Build failed in Jenkins: kafka-1.1-jdk7 #5

2018-02-02 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Update docs for KIP-229 (#4499) -- [...truncated 410.92 KB...] kafka.zookeeper.ZooKeeperClientTest > testPipelinedGetData STARTED kafka.zookeeper.

[jira] [Created] (KAFKA-6523) kafka server not starting

2018-02-02 Thread Sanjeevani Mehra (JIRA)
Sanjeevani Mehra created KAFKA-6523: --- Summary: kafka server not starting Key: KAFKA-6523 URL: https://issues.apache.org/jira/browse/KAFKA-6523 Project: Kafka Issue Type: Bug Rep

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

2018-02-02 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Update docs for KIP-229 (#4499) -- [...truncated 411.46 KB...] kafka.zk.KafkaZkClientTest > testPropagateIsrChanges STARTED kafka.zk.KafkaZkC