[jira] [Created] (KAFKA-7236) Add --critical-partitions option to describe topics command

2018-08-02 Thread Kevin Lu (JIRA)
Kevin Lu created KAFKA-7236: --- Summary: Add --critical-partitions option to describe topics command Key: KAFKA-7236 URL: https://issues.apache.org/jira/browse/KAFKA-7236 Project: Kafka Issue Type:

[DISCUSS] KIP-351: Add --critical-partitions option to describe topics command

2018-08-02 Thread Kevin Lu
Hi friends! This thread is to discuss KIP-351 ! I am proposing to add a --critical-partitions option to the describe topics command that will only list out topic partitions t

Re: [DISCUSS] KIP-351: Add --critical-partitions option to describe topics command

2018-08-02 Thread Mickael Maison
What about also adding a --under-minisr-partitions option? That would match the "kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount" broker metric and it's usually pretty relevant when investigating issues On Thu, Aug 2, 2018 at 8:54 AM, Kevin Lu wrote: > Hi friends! > > This thread

Re: [DISCUSSION] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-02 Thread Viktor Somogyi
Hi Chia-Ping, Sorry for the delay on this. One thought though: looking at current implementations on github they seemed a bit elaborate, which makes me think that people don't usually want to use it as a lambda. But since in your KIP you added it, what was your use case there? "Q: Which implement

Re: [DISCUSS] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-08-02 Thread Stanislav Kozlovski
Hi group, I've updated the KIP and PR with the discussed interface changes. I am also starting a voting thread Best, Stanislav On Thu, Aug 2, 2018 at 1:27 AM Jason Gustafson wrote: > Hey Stanislav, > > Just to make sure I understood you right - you propose not exposing any new > > exception ty

[jira] [Created] (KAFKA-7237) Add explicit fatal marker to fatal error messages

2018-08-02 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-7237: - Summary: Add explicit fatal marker to fatal error messages Key: KAFKA-7237 URL: https://issues.apache.org/jira/browse/KAFKA-7237 Project: Kafka Issue Type:

[VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-08-02 Thread Stanislav Kozlovski
Hey everybody, I'd like to start a vote thread for KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation -- Best, Stanislav

Re: [Vote] KIP-321: Update TopologyDescription to better represent Source and Sink Nodes

2018-08-02 Thread Damian Guy
You have 3 binding votes, so i'll defer to the others. On Thu, 2 Aug 2018 at 04:41 Nishanth Pradeep wrote: > The only issue I see with this is that Sink#topic would also need to be > Optional as was pointed out already. Since Sink#topic is a preexisting > method, changing its return type would b

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

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

[jira] [Created] (KAFKA-7238) Producer can't update Metadata when two brokers break down and restart the first one

2018-08-02 Thread little brother ma (JIRA)
little brother ma created KAFKA-7238: Summary: Producer can't update Metadata when two brokers break down and restart the first one Key: KAFKA-7238 URL: https://issues.apache.org/jira/browse/KAFKA-7238

Re: [VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-08-02 Thread Ted Yu
+1 Original message From: Stanislav Kozlovski Date: 8/2/18 2:41 AM (GMT-08:00) To: dev@kafka.apache.org Subject: [VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation Hey everybody, I'd like to start a vote thread for KIP-3

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

2018-08-02 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Implement toString() in config validator classes (#5401) -- [...truncated 1.98 MB...] org.apache.kafka.streams.StreamsConfigTest > shou

Re: [Vote] KIP-321: Update TopologyDescription to better represent Source and Sink Nodes

2018-08-02 Thread Guozhang Wang
I think leaving the current return value to be null-able is okay, as long as it is well documented in java doc. Guozhang On Thu, Aug 2, 2018 at 3:13 AM, Damian Guy wrote: > You have 3 binding votes, so i'll defer to the others. > > On Thu, 2 Aug 2018 at 04:41 Nishanth Pradeep > wrote: > > > T

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

2018-08-02 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-08-02 Thread Jason Gustafson
Hey Stanislav, I should have noticed it earlier from your example, but I just realized that interfaces don't mix well with exceptions. There is no way to catch the interface type, which means you have to depend on instanceof checks, which is not very conventional. At the moment, we raise Serializa

Re: [DISCUSS] KIP-351: Add --critical-partitions option to describe topics command

2018-08-02 Thread Kevin Lu
Hi Mickael, Thanks for the suggestion! Correct me if I am mistaken, but if a producer attempts to send to a partition that is under min ISR (and ack=all or -1) then the send will fail with a NotEnoughReplicas or NotEnoughReplicasAfterAppend exception? At this point, client-side has already suffer

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

2018-08-02 Thread Colin McCabe
On Wed, Aug 1, 2018, at 11:35, James Cheng wrote: > I’m a little confused about something. Is this KIP focused on log > cleaner exceptions in general, or focused on log cleaner exceptions due > to disk failures? > > Will max.uncleanable.partitions apply to all exceptions (including log > cleane

Re: [Vote] KIP-321: Update TopologyDescription to better represent Source and Sink Nodes

2018-08-02 Thread Matthias J. Sax
I agree with Guozhang. Breaking compatibility is not acceptable. If we want the change to use `Optional`, we should deprecate the current method and explain that it return type will change in next major release 3.0.0 and create a ticket for this change that we can tackle when time comes. -Matt

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

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

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

2018-08-02 Thread Colin McCabe
Thanks, Jason. I don't have a very strong opinion on this. But like you said, if we skip bumping the RPC versions, this would be a smaller change, which might be good. best, Colin On Wed, Aug 1, 2018, at 17:43, Jason Gustafson wrote: > Hey Vahid, > > I talked with Colin offline. I think spe

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

2018-08-02 Thread Colin McCabe
+1 for starting the vote cheers, Colin On Wed, Aug 1, 2018, at 08:46, Manikumar wrote: > Hi all, > > If there are no concerns, I will start the voting process soon. > > Thanks > > On Tue, Jul 31, 2018 at 9:08 AM Manikumar wrote: > > > Hi Colin, > > > > Yes, "--authorizer-properties" option

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

2018-08-02 Thread Ray Chiang
I see this as a fix for the LogCleaner.  Uncaught exceptions kill the CleanerThread and this is viewed as undesired behavior.  Some other ways to think of this fix: 1) If you have occasional corruption in some log segments, then with each broker restart, the LogCleaner will lose its state, re-

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

2018-08-02 Thread Ray Chiang
One more thing occurred to me.  Should the configuration property be named "max.uncleanable.partitions.per.disk" instead? -Ray On 8/1/18 9:11 AM, Stanislav Kozlovski wrote: Yes, good catch. Thank you, James! Best, Stanislav On Wed, Aug 1, 2018 at 5:05 PM James Cheng wrote: Can you update

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

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

KIP-352: Distinguish URPs caused by reassignment

2018-08-02 Thread Jason Gustafson
Hey All, Another day, another KIP. This one is hopefully straightforward: https://cwiki.apache.org/confluence/display/KAFKA/KIP-352%3A+Distinguish+URPs+caused+by+reassignment. Have a look and let me know what you think! Thanks, Jason

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

2018-08-02 Thread Vahid S Hashemian
Hi Jason, Colin, Thanks for your feedback. If we are going to keep supporting the empty group id like before then it makes sense to do what you suggested. Though, IMHO if using the empty group id for committing offsets is a bad practice we should stop supporting it at some point; and it would

Re: KIP-352: Distinguish URPs caused by reassignment

2018-08-02 Thread Ismael Juma
Thanks Jason. This is definitely a pain point. I actually prefer the option to redefine what under-replicated means (currently under rejected alternatives). Also, do we need to make changes to what we store in ZK? If so, that should be in the KIP too. Ismael On Thu, Aug 2, 2018 at 11:45 AM Jason

[jira] [Created] (KAFKA-7239) Kafka Connect secret externalization not working

2018-08-02 Thread satyanarayan komandur (JIRA)
satyanarayan komandur created KAFKA-7239: Summary: Kafka Connect secret externalization not working Key: KAFKA-7239 URL: https://issues.apache.org/jira/browse/KAFKA-7239 Project: Kafka

Re: Discussion: New components in JIRA?

2018-08-02 Thread Guozhang Wang
Hello Ray, I've added these two components. People should be able to use them creating / updating the JIRAs now. Guozhang On Wed, Aug 1, 2018 at 12:56 PM, Ray Chiang wrote: > I haven't seen any comments. Let me know if/when you add the new > components. Thanks. > > -Ray > > > > On 7/27/18 9

Re: Discussion: New components in JIRA?

2018-08-02 Thread Ray Chiang
Great.  Thanks! -Ray On 8/2/18 12:28 PM, Guozhang Wang wrote: Hello Ray, I've added these two components. People should be able to use them creating / updating the JIRAs now. Guozhang On Wed, Aug 1, 2018 at 12:56 PM, Ray Chiang wrote: I haven't seen any comments. Let me know if/when you

[VOTE] KIP-341: Update Sticky Assignor's User Data Protocol

2018-08-02 Thread Vahid S Hashemian
Hi everyone, I believe the feedback on this KIP has been addressed so far. So I'd like to start a vote. The KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-341%3A+Update+Sticky+Assignor%27s+User+Data+Protocol Discussion thread: https://www.mail-archive.com/dev@kafka.apache.org/msg897

[jira] [Created] (KAFKA-7240) -total metrics in Streams are incorrect

2018-08-02 Thread Sam Lendle (JIRA)
Sam Lendle created KAFKA-7240: - Summary: -total metrics in Streams are incorrect Key: KAFKA-7240 URL: https://issues.apache.org/jira/browse/KAFKA-7240 Project: Kafka Issue Type: Bug Com

ConsumerGroupCommand tool improvement?

2018-08-02 Thread Vahid S Hashemian
Hi all, A requirement has been raised by a colleague and I wanted to see if there is any interest in the community in adding the functionality to Apache Kafka. ConsumerGroupCommand tool in describe ('--describe' or '--describe --offsets') mode currently lists all topics the group has consumed

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

2018-08-02 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7231; Ensure NetworkClient uses overridden request timeout (#5444) -- [...truncated 1.97 MB...] org.apache.kafka.streams.processor.internals.I

[jira] [Resolved] (KAFKA-7125) Calling StreamsBuilderbuilder.build().describe() causes java.util.NoSuchElementException: null

2018-08-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7125?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-7125. -- Resolution: Won't Fix > Calling StreamsBuilderbuilder.build().describe() causes > java.util.No

[jira] [Resolved] (KAFKA-7049) InternalTopicIntegrationTest sometimes fails

2018-08-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7049?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-7049. -- Resolution: Cannot Reproduce > InternalTopicIntegrationTest sometimes fails > -

[jira] [Resolved] (KAFKA-7228) DeadLetterQueue throws a NullPointerException

2018-08-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7228?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7228. -- Resolution: Fixed > DeadLetterQueue throws a NullPointerException > ---

[jira] [Resolved] (KAFKA-4101) java.lang.IllegalStateException in org.apache.kafka.common.network.Selector.channelOrFail

2018-08-02 Thread Ray Chiang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ray Chiang resolved KAFKA-4101. --- Resolution: Duplicate > java.lang.IllegalStateException in > org.apache.kafka.common.network.Selecto

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

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

Re: [VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-08-02 Thread Stanislav Kozlovski
Hi Jason and Ted, @Jason I did not oppose the idea as I'm not too familiar with Java conventions. I agree it is a non-ideal way for the user to catch the exception so I changed it back. I've updated the KIP and PR with the latest changes. Now, there is only one public exception `FaultyRecordExcep

Jenkins build is back to normal : kafka-2.0-jdk8 #99

2018-08-02 Thread Apache Jenkins Server
See

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

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

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

2018-08-02 Thread Apache Jenkins Server
See Changes: [me] KAFKA-7228: Set errorHandlingMetrics for dead letter queue -- [...truncated 433.72 KB...] kafka.controller.PartitionStateMachineTest > testInvalidNonexist

Re: [VOTE] KIP-342 - Add support for custom SASL extensions in OAuthBearer authentication

2018-08-02 Thread Jun Rao
Hi, Stanislav, Thanks for the KIP. +1 Just one minor comment. Since the JWT token supports customizable claim fields, it would be useful to clarify when to use the SASL extension vs the customized fields in JWT. Jun On Wed, Jul 25, 2018 at 10:03 AM, Stanislav Kozlovski < stanis...@confluent.io>

Jenkins build is back to normal : kafka-trunk-jdk10 #360

2018-08-02 Thread Apache Jenkins Server
See

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

2018-08-02 Thread Manikumar
Hi All, I would like to start voting on KIP-332 which allows AclCommand to use AdminClient API for acl management. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-332%3A+Update+AclCommand+to+use+AdminClient+API Discussion Thread: https://www.mail-archive.com/dev@kafka.apache.org/msg90

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

2018-08-02 Thread Ted Yu
+1 On Thu, Aug 2, 2018 at 7:33 PM Manikumar wrote: > Hi All, > > I would like to start voting on KIP-332 which allows AclCommand to use > AdminClient API for acl management. > > KIP: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-332%3A+Update+AclCommand+to+use+AdminClient+API > > Dis

[jira] [Resolved] (KAFKA-7233) InMemoryKeyValueStore is not thread-safe for Interactive Queries

2018-08-02 Thread Hashan Gayasri Udugahapattuwa (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hashan Gayasri Udugahapattuwa resolved KAFKA-7233. -- Resolution: Invalid The implementation is correct > InMemoryKe

[jira] [Resolved] (KAFKA-7083) maxTickMessages in ConsumerGroup option

2018-08-02 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7083?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-7083. -- Resolution: Information Provided Closing the issue related to node.js client. Please contact client lib

[jira] [Resolved] (KAFKA-7226) kafka-console-consumer.sh doesn't use security.protocol provided in config file

2018-08-02 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-7226. -- Resolution: Information Provided Closing the issue related to HDP Kafka distribution. > kafka-console-

Re: [DISCUSS] KIP-351: Add --critical-partitions option to describe topics command

2018-08-02 Thread Jason Gustafson
Hey Kevin, Thanks for the KIP. I like Mickael's suggestion to add --under-minisr-partitions since it fits with the metric we already expose. It's also a good question whether there should be a separate category for partitions which are right at min.isr. I'm reluctant to add new categories, but I a

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

2018-08-02 Thread Apache Jenkins Server
See Changes: [github] KAFKA-3514: Part II, Choose tasks with data on all partitions to process -- [...truncated 2.47 MB...] org.apache.kafka.streams.TopologyTest > multi

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

2018-08-02 Thread Jason Gustafson
Hey Vahid, I think we're in agreement. Perhaps since none of us feel too strongly, we should go for the more minimal change? I'll vote for the KIP either way as long as we can change the default group.id. Thanks, Jason On Thu, Aug 2, 2018 at 11:55 AM, Vahid S Hashemian < vahidhashem...@us.ibm.co