Re: [VOTE] KIP-584: Versioning scheme for features

2020-04-24 Thread Colin McCabe
On Fri, Apr 24, 2020, at 00:01, Kowshik Prakasam wrote: > (Kowshik): Great point! However for case #1, I'm not sure why we need to > create a '/features' ZK node with disabled features. Instead, do you see > any drawback if we just do not create it? i.e. if IBP is less than 2.6, the > controller

Re: [Vote] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-24 Thread Boyang Chen
Thanks a lot John for giving a review on Friday night! And thank you Guozhang, and Jason for your votes as well :) Now that we have collected 3 binding votes (Guozhang, Jason, John), I will close the voting thread and mark the KIP as approved. Still feel free to raise any question on the mailing

Re: [Vote] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-24 Thread John Roesler
Hi Boyang, Thanks for the KIP! I've just read it over and caught up on all the prior discussions. The current version of the KIP looks good to me, and I think the decisions you've made are reasonable. I'm +1 (binding) Thanks, -John On Wed, Apr 22, 2020, at 12:12, Boyang Chen wrote: > Hey

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-04-24 Thread John Roesler
Hi Guozhang, Thanks for the KIP! I took a quick look, and I'm really happy to see this underway. Some quick questions: 1. Can you elaborate on the reason that stores just have a list of serdes, whereas other components have an explicit key/value serde? 1.5. A side-effect of this seems to be

[jira] [Resolved] (KAFKA-9853) Improve performance of Log.fetchOffsetByTimestamp()

2020-04-24 Thread Eric Bolinger (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eric Bolinger resolved KAFKA-9853. -- Fix Version/s: 2.6.0 Reviewer: Ismael Juma Resolution: Implemented PR# 8474

Build failed in Jenkins: kafka-trunk-jdk14 #18

2020-04-24 Thread Apache Jenkins Server
See Changes: [github] HOTFIX: Fix broker bounce system tests (#8532) -- [...truncated 6.06 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

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

2020-04-24 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-9917) Consumer not assigned any partitions in group after broker restart

2020-04-24 Thread Igor Baltiyskiy (Jira)
Igor Baltiyskiy created KAFKA-9917: -- Summary: Consumer not assigned any partitions in group after broker restart Key: KAFKA-9917 URL: https://issues.apache.org/jira/browse/KAFKA-9917 Project: Kafka

Re: [Discuss] KIP-582 Add a "continue" option for Kafka Connect error handling

2020-04-24 Thread Zihan Li
Hi Chris, Thanks a lot for your comments. 1. The complexity comes from maintaining an additional topic and a connector, rather than configuring them. Users need to spend extra time and money to maintain the additional connectors. I can imagine a case where a user has 3 topics consumed by S3,

Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-24 Thread Andrew Schofield
I wonder whether we're getting a bit overcomplicated with this. I think all that's required here is to add an optional guard predicate for a Transformation. The predicate cannot end the Transformation chain, but it can allow or prevent a particular Transformation from running. How about this as

Re: KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-04-24 Thread Jun Rao
Hi, David, Thanks for the KIP. A few quick comments. 1. About quota.partition.mutations.rate. I am not sure if it's very easy for the user to set the quota as a rate. For example, each partition mutation could take a different number of ZK operations (depending on things like retry). The time to

Build failed in Jenkins: kafka-trunk-jdk11 #1387

2020-04-24 Thread Apache Jenkins Server
See Changes: [github] HOTFIX: Fix broker bounce system tests (#8532) -- [...truncated 3.03 MB...] org.apache.kafka.streams.TestTopicsTest > testKeyValuesToMap PASSED

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

2020-04-24 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9704: Fix the issue z/OS won't let us resize file when mmap. -- [...truncated 6.02 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [Discuss] KIP-582 Add a "continue" option for Kafka Connect error handling

2020-04-24 Thread Zihan Li
Hi Chris, Thanks a lot for your comments. The complexity comes from maintaining an additional topic and a connector, rather than configuring them. Users need to spend extra time and money to maintain the additional connectors. I can imagine a case where a user has 3 topics consumed by S3,

[jira] [Created] (KAFKA-9916) Materialize Table-Table Join Result to Avoid Performing Same Join Twice

2020-04-24 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9916: Summary: Materialize Table-Table Join Result to Avoid Performing Same Join Twice Key: KAFKA-9916 URL: https://issues.apache.org/jira/browse/KAFKA-9916 Project: Kafka

KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-04-24 Thread David Jacot
Hi folks, I'd like to start the discussion for KIP-599: https://cwiki.apache.org/confluence/display/KAFKA/KIP-599%3A+Throttle+Create+Topic%2C+Create+Partition+and+Delete+Topic+Operations It proposes to introduce quotas for the create topics, create partitions and delete topics operations. Let me

Build failed in Jenkins: kafka-trunk-jdk11 #1386

2020-04-24 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9704: Fix the issue z/OS won't let us resize file when mmap. -- [...truncated 3.03 MB...] org.apache.kafka.streams.TestTopicsTest >

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-04-24 Thread Stanislav Kozlovski
Thanks for the KIP, Gokul! I like the overall premise - I think it's more user-friendly to have configs for this than to have users implement their own config policy -> so unless it's very complex to implement, it seems worth it. I agree that having the topic policy on the CreatePartitions path

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-04-24 Thread Ismael Juma
Right, some companies run them nightly. What I meant to ask is if we changed the configuration so that TLS 1.3 is exercised in the system tests by default. Ismael On Fri, Apr 24, 2020 at 7:32 AM Nikolay Izhikov wrote: > Hello, Ismael. > > AFAIK we don’t run system tests nightly. > Do we have

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-04-24 Thread Nikolay Izhikov
Hello, Ismael. AFAIK we don’t run system tests nightly. Do we have resources to run system tests periodically? When I did the testing I used servers my employer gave me. > 24 апр. 2020 г., в 08:05, Ismael Juma написал(а): > > Hi Nikolay, > > Seems like we have been able to run the system

[jira] [Resolved] (KAFKA-9704) z/OS won't let us resize file when mmap

2020-04-24 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9704?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-9704. --- Resolution: Fixed > z/OS won't let us resize file when mmap >

[jira] [Created] (KAFKA-9915) Throttle Create Topic, Create Partition and Delete Topic Operations

2020-04-24 Thread David Jacot (Jira)
David Jacot created KAFKA-9915: -- Summary: Throttle Create Topic, Create Partition and Delete Topic Operations Key: KAFKA-9915 URL: https://issues.apache.org/jira/browse/KAFKA-9915 Project: Kafka

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-04-24 Thread Gokul Ramanan Subramanian
Hi Tom. With KIP-578, we are not trying to model the load on each partition, and come up with an exact limit on what the cluster or broker can handle in terms of number of partitions. We understand that not all partitions are equal, and the actual load per partition varies based on the message

[jira] [Created] (KAFKA-9914) Mirror Maker 2 creates heartbeats kafka topics recursively

2020-04-24 Thread azher khan (Jira)
azher khan created KAFKA-9914: - Summary: Mirror Maker 2 creates heartbeats kafka topics recursively Key: KAFKA-9914 URL: https://issues.apache.org/jira/browse/KAFKA-9914 Project: Kafka Issue

Jenkins build is back to normal : kafka-trunk-jdk11 #1385

2020-04-24 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-24 Thread Sönke Liebau
After thinking on this a little bit, maybe this would be an option: add default methods serialize and deserialize to the KafkaPrincipalBuilder interface, these could be very short: default String serialize(KafkaPrincipal principal) { return principal.toString(); } default KafkaPrincipal

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

2020-04-24 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8639: Replace AddPartitionsToTxn with Automated Protocol (#8326) -- [...truncated 3.01 MB...]

Re: [VOTE] KIP-584: Versioning scheme for features

2020-04-24 Thread Kowshik Prakasam
Hi Colin, Thanks a lot for the excellent feedback and great ideas/questions/suggestions. I have updated the KIP based on the feedback. Please find my response below to the comments. > It would be good to note that deprecating a feature version (in other words, increasing minVersionLevel on the

[jira] [Reopened] (KAFKA-9859) kafka-streams-application-reset tool doesn't take into account topics generated by KTable foreign key join operation

2020-04-24 Thread Levani Kokhreidze (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9859?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Levani Kokhreidze reopened KAFKA-9859: -- > kafka-streams-application-reset tool doesn't take into account topics > generated by

Build failed in Jenkins: kafka-trunk-jdk11 #1384

2020-04-24 Thread Apache Jenkins Server
See Changes: [github] MINOR: equals() should compare all fields for generated classes (#8539) -- [...truncated 3.03 MB...]