Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-23 Thread Jason Gustafson
Hey Vahid, Thanks for the updates. Just to clarify, I was suggesting that we disable enable.auto.commit only if no explicit group.id is configured. If an explicit empty string is configured for the group id, then maybe we keep the current behavior for compatibility. We can log a warning

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Lucas Wang
Thanks for the comment, Becket. So far, we've been trying to avoid making any request handler thread special. But if we were to follow that path in order to make the two planes more isolated, what do you think about also having a dedicated processor thread, and dedicated port for the controller?

Re: [DISCUSS] KIP-346 - Limit blast radius of log compaction failure

2018-07-23 Thread Stanislav Kozlovski
Hi Ted, Yes, absolutely. Thanks for pointing that out! On Mon, Jul 23, 2018 at 6:12 PM Ted Yu wrote: > For `uncleanable-partitions`, should the example include topic name(s) ? > > Cheers > > On Mon, Jul 23, 2018 at 5:46 PM Stanislav Kozlovski < > stanis...@confluent.io> > wrote: > > > I

Re: [Discuss] KIP-321: Add method to get TopicNameExtractor in TopologyDescription

2018-07-23 Thread Nishanth Pradeep
Sounds good to me too. As far as deprecating goes -- should the topics() method removed completely or should it have a @deprecated annotation for removal in some future version? Best, Nishanth Pradeep On Sun, Jul 22, 2018 at 1:32 PM Matthias J. Sax wrote: > Works for me. > > On 7/22/18 9:48

Re: Seeing old tombstones in compacted topic

2018-07-23 Thread Ted Yu
Looking at some recent JIRAs, such as KAFKA-6568, which came in after the release of 0.11.0 Would that possibly be related to what you observed ? Cheers On Mon, Jul 23, 2018 at 6:23 PM Mitch Seymour wrote: > Hi all, > > We're using version 0.11.0 of Kafka (broker and client), and our Kafka >

Seeing old tombstones in compacted topic

2018-07-23 Thread Mitch Seymour
Hi all, We're using version 0.11.0 of Kafka (broker and client), and our Kafka Streams app uses a compacted topic for storing it's state. Here's the output of kafka-topics.sh --describe: Topic:mytopic PartitionCount:32 ReplicationFactor:2 Configs:retention.ms=43200,cleanup.policy=compact

Re: [DISCUSS] KIP-346 - Limit blast radius of log compaction failure

2018-07-23 Thread Ted Yu
For `uncleanable-partitions`, should the example include topic name(s) ? Cheers On Mon, Jul 23, 2018 at 5:46 PM Stanislav Kozlovski wrote: > I renamed the KIP and that changed the link. Sorry about that. Here is the > new link: > >

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Becket Qin
Personally I am not fond of the dequeue approach simply because it is against the basic idea of isolating the controller plane and data plane. With a single dequeue, theoretically speaking the controller requests can starve the clients requests. I would prefer the approach with a separate

Re: [DISCUSS] KIP-346 - Limit blast radius of log compaction failure

2018-07-23 Thread Stanislav Kozlovski
I renamed the KIP and that changed the link. Sorry about that. Here is the new link: https://cwiki.apache.org/confluence/display/KAFKA/KIP-346+-+Improve+LogCleaner+behavior+on+error On Mon, Jul 23, 2018 at 5:11 PM Stanislav Kozlovski wrote: > Hey group, > > I created a new KIP about making log

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Lucas Wang
Sure, I can summarize the usage of correlation id. But before I do that, it seems the same out-of-order processing can also happen to Produce requests sent by producers, following the same example you described earlier. If that's the case, I think this probably deserves a separate doc and design

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-23 Thread Stanislav Kozlovski
Hey Vahid, No I don't see an issue with it. I believe it to be the best approach. Best, Stanisav On Mon, Jul 23, 2018 at 12:41 PM Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Hi Stanislav, > > Thanks for the feedback. > Do you see an issue with using `null` as the default group id

[DISCUSS] KIP-346 - Limit blast radius of log compaction failure

2018-07-23 Thread Stanislav Kozlovski
Hey group, I created a new KIP about making log compaction more fault-tolerant. Please give it a look here and please share what you think, especially in regards to the points in the "Needs Discussion" paragraph. KIP: KIP-346

Build failed in Jenkins: kafka-2.0-jdk8 #88

2018-07-23 Thread Apache Jenkins Server
See -- [...truncated 2.48 MB...] org.apache.kafka.streams.TopologyTest > shouldNotAllowToAddStateStoreToNonExistingProcessor STARTED org.apache.kafka.streams.TopologyTest >

Re: Discussion: New components in JIRA?

2018-07-23 Thread Guozhang Wang
I've just updated the web docs on http://kafka.apache.org/contributing accordingly. On Mon, Jul 23, 2018 at 3:30 PM, khaireddine Rezgui < khaireddine...@gmail.com> wrote: > Good job Ray for the wiki, it's clear enough. > > Le 23 juil. 2018 10:17 PM, "Ray Chiang" a écrit : > > Okay, I've created

[jira] [Created] (KAFKA-7196) Remove heartbeat delayed operation for those removed consumers at the end of each rebalance

2018-07-23 Thread Lincong Li (JIRA)
Lincong Li created KAFKA-7196: - Summary: Remove heartbeat delayed operation for those removed consumers at the end of each rebalance Key: KAFKA-7196 URL: https://issues.apache.org/jira/browse/KAFKA-7196

Re: Discussion: New components in JIRA?

2018-07-23 Thread khaireddine Rezgui
Good job Ray for the wiki, it's clear enough. Le 23 juil. 2018 10:17 PM, "Ray Chiang" a écrit : Okay, I've created a wiki page Reporting Issues in Apache Kafka < https://cwiki.apache.org/confluence/display/KAFKA/Reporting+Issues+in+Apache+Kafka>. I'd appreciate any feedback. If this is good

Build failed in Jenkins: kafka-trunk-jdk10 #310

2018-07-23 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7193: Use ZooKeeper IP address in streams tests to avoid timeouts -- [...truncated 1.54 MB...] kafka.admin.TopicCommandTest >

Re: Discussion: New components in JIRA?

2018-07-23 Thread Ray Chiang
Okay, I've created a wiki page Reporting Issues in Apache Kafka .  I'd appreciate any feedback.  If this is good enough, I can file a JIRA to change the link under "Bugs" in the "Project information" page.

[jira] [Created] (KAFKA-7195) StreamStreamJoinIntegrationTest fails in 2.0 Jenkins

2018-07-23 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7195: - Summary: StreamStreamJoinIntegrationTest fails in 2.0 Jenkins Key: KAFKA-7195 URL: https://issues.apache.org/jira/browse/KAFKA-7195 Project: Kafka Issue Type: Test

Build failed in Jenkins: kafka-2.0-jdk8 #87

2018-07-23 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-7193: Use ZooKeeper IP address in streams tests to avoid timeouts -- [...truncated 2.48 MB...] org.apache.kafka.streams.TopologyTest >

Request for permission to create KIP

2018-07-23 Thread Afshartous, Nick
Hi all, Requesting permission to create a KIP in regards to KAFKA-6690 Priorities for Source Topics My Wiki ID is nafshartous. Cheers, -- Nick

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-23 Thread Vahid S Hashemian
Hi Stanislav, Thanks for the feedback. Do you see an issue with using `null` as the default group id (as addressed by Jason in his response)? This default group id would not support offset commits and consumers would use `auto.offset.reset` config when there is no current offset. Thanks.

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Dong Lin
Hey Lucas, Could you update the KIP if you are confident with the approach which uses correlation id? The idea around correlation id is kind of scattered across multiple emails. It will be useful if other reviews can read the KIP to understand the latest proposal. Thanks, Dong On Mon, Jul 23,

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-23 Thread Vahid S Hashemian
Hi Dhruvil, Could you please share the reasoning behind your suggestion? My understanding is that there is more restrictions around topic names because they appear in folder names, and there are limitations by file systems on what can be used in file/folder names or how long they can be.

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-23 Thread Mayuresh Gharat
I like the idea of the dequeue implementation by Lucas. This will help us avoid additional queue for controller and additional configs in Kafka. Thanks, Mayuresh On Sun, Jul 22, 2018 at 2:58 AM Becket Qin wrote: > Hi Jun, > > The usage of correlation ID might still be useful to address the

Build failed in Jenkins: kafka-trunk-jdk10 #309

2018-07-23 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H20 (ubuntu xenial) in workspace

1.1.1 and KAFKA-6929

2018-07-23 Thread Jordan Pilat
It looks like even though the fix for this¹ was backported to the 1.1 line² as part of the PR³, the ticket didn't get updated to include a fix version of 1.1.1, and thus missed inclusion in the 1.1.1 release notes. 1 - https://issues.apache.org/jira/browse/KAFKA-6929 2 -

Re: Discussion: New components in JIRA?

2018-07-23 Thread Ray Chiang
Good point.  I'll look into adding some JIRA guidelines to the documentation/wiki. -Ray On 7/22/18 10:23 AM, Guozhang Wang wrote: Hello Ray, Thanks for brining this up. I'm generally +1 on the first two, while for the last category, personally I felt leaving them as part of `tools` is fine,

Re: [DISCUSS] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-07-23 Thread Vahid S Hashemian
Hi Jason, Thanks for the feedback. 1. I think the current KIP follows your first bullet suggestion; except for the addition of a OffsetCommit version and continuing to support old versions (I will add these). 2. Regarding your suggestion, using `null` as default group id seems reasonable.

[jira] [Resolved] (KAFKA-7193) ZooKeeper client times out with localhost due to random choice of ipv4/ipv6

2018-07-23 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7193. --- Resolution: Fixed Reviewer: Ismael Juma Fix Version/s: 2.0.0 > ZooKeeper

[jira] [Resolved] (KAFKA-6918) Kafka server fails to start with IBM JAVA

2018-07-23 Thread Ray Chiang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ray Chiang resolved KAFKA-6918. --- Resolution: Fixed Fix Version/s: 1.1.1 > Kafka server fails to start with IBM JAVA >

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-23 Thread Ron Dagostino
Hi Rajini. I think a class is fine as long as we make sure the semantics of immutability are clear -- it would have to be a value class, and any constructor that accepts a Map as input would have to copy that Map rather than store it in a member variable. Similarly, any Map that it might return

[jira] [Created] (KAFKA-7194) Error deserializing assignment after rebalance

2018-07-23 Thread Konstantine Karantasis (JIRA)
Konstantine Karantasis created KAFKA-7194: - Summary: Error deserializing assignment after rebalance Key: KAFKA-7194 URL: https://issues.apache.org/jira/browse/KAFKA-7194 Project: Kafka

Re: Question about issues of Kafka release version 1.1.1

2018-07-23 Thread Ismael Juma
Seems like you're right Lambdaliu. Rajini/Jason, can you please check and update the JIRAs? Ismael On Mon, Jul 23, 2018 at 7:09 AM lambdaliu(刘少波) wrote: > Hi team, > > I Have downloaded the source release of kafka version 1.1.1 and found the > JIRA > issues KAFKA-6911 and KAFKA-6809 listed in

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-23 Thread Rajini Sivaram
Hi Ron, Stanislav, I agree with Stanislav that it would be better to leave `SaslExtensions` as a class rather than make it an interface. We don''t really expect users to extends this class, so it is convenient to have an implementation since users need to create an instance. The class provided by

[DISCUSS] KIP-332: Update AclCommand to use AdminClient API

2018-07-23 Thread Manikumar
Hi all, I have created a KIP to use AdminClient API in AclCommand (kafka-acls.sh) *https://cwiki.apache.org/confluence/display/KAFKA/KIP-332%3A+Update+AclCommand+to+use+AdminClient+API*

Re: Plan for new Kafka Connect Transform

2018-07-23 Thread Randall Hauch
How about changing the ReplaceField SMT to be able to support nested fields? If we come up with a unified way to identify nested fields, then we could add support for nested fields to other SMTs, too. Best regards, Randall On Thu, Jul 19, 2018 at 2:18 PM, karri saisatish kumar reddy <

Question about issues of Kafka release version 1.1.1

2018-07-23 Thread 刘少波
Hi team, I Have downloaded the source release of kafka version 1.1.1 and found the JIRA issues KAFKA-6911 and KAFKA-6809 listed in the release notes but it's PR looks like doesn't contain in the source release. Is this a valid situation? Should we create a JIRA issue to trace it? Regards,

KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-07-23 Thread Adam Bellemare
Here is the new discussion thread for KIP-213. I picked back up on the KIP as this is something that we too at Flipp are now running in production. Jan started this last year, and I know that Trivago is also using something similar in production, at least in terms of APIs and functionality.

[jira] [Created] (KAFKA-7193) ZooKeeper client times out with localhost due to random choice of ipv4/ipv6

2018-07-23 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-7193: - Summary: ZooKeeper client times out with localhost due to random choice of ipv4/ipv6 Key: KAFKA-7193 URL: https://issues.apache.org/jira/browse/KAFKA-7193 Project:

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-23 Thread Ron Dagostino
Hi Stanislav. See https://tools.ietf.org/html/rfc7628#section-3.1, and that section refers to the core ABNF productions defined in https://tools.ietf.org/html/rfc5234#appendix-B. Ron > On Jul 23, 2018, at 1:30 AM, Stanislav Kozlovski > wrote: > > Hey Ron and Rajini, > > Here are my

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

2018-07-23 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-7192) State-store can desynchronise with changelog

2018-07-23 Thread Jon Bates (JIRA)
Jon Bates created KAFKA-7192: Summary: State-store can desynchronise with changelog Key: KAFKA-7192 URL: https://issues.apache.org/jira/browse/KAFKA-7192 Project: Kafka Issue Type: Bug