Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread David Jacot
Hi all, First, I agree with what has been discussed. Having 3.x as the bridge releases and entirely removing ZK in 4.0 makes total sense. Second, what would you think about removing the auto topics creation in 3.0? It is not recommended to use it anymore and that could simplify a bit our path tow

[jira] [Created] (KAFKA-9978) Improve poll(Duration) and KafkaConsumer Javadoc

2020-05-11 Thread Benoit Delbosc (Jira)
Benoit Delbosc created KAFKA-9978: - Summary: Improve poll(Duration) and KafkaConsumer Javadoc Key: KAFKA-9978 URL: https://issues.apache.org/jira/browse/KAFKA-9978 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-585: Conditional SMT

2020-05-11 Thread Andrew Schofield
Hi, I have implemented some of this and configured some predicates and I prefer this slight modification of the ? syntax: transforms: t2 transforms.t2?type: org.apache.kafka.connect.transforms.predicates.TopicNameMatch transforms.t2?negate: true transforms.t2?pattern: my-prefix-.* transforms.t2.

Re: [DISCUSS] KIP-585: Conditional SMT

2020-05-11 Thread Tom Bentley
Hi Andrew, That works nicely enough for the proposal where the predicate is configured directly on the transformation. But I thought there was more consensus around the proposal to have the transformation configuration refer to a predicate indirectly, defined with the ?predicates key. I guess an e

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

2020-05-11 Thread David Jacot
Hi Anna and Jun, Anna, thanks for your thoughtful feedback. Overall, I agree with what you said. If I summarize, you said that using time on server threads is not easier to tune than a rate based approach and it does not really capture all the load neither as the control requests are not taken int

Re: [VOTE] KIP-605 - Expand Connect Worker Internal Topic Settings

2020-05-11 Thread Randall Hauch
Ping for reviewers. I guess I never voted, so +1 (binding). On Thu, May 7, 2020 at 4:13 PM Christopher Egerton wrote: > LGTM, +1 (non-binding) > > Thanks Randall! > > On Thu, May 7, 2020 at 11:01 AM Randall Hauch wrote: > > > I'd like to open the vote for KIP-605: > > > > > https://cwiki.apach

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

2020-05-11 Thread David Jacot
Hi Tom, Thanks for the feedback. >> If I understand the proposed throttling algorithm, an initial request would >> be allowed (possibly making K negative) and only subsequent requests >> (before K became positive) would receive the QUOTA_VIOLATED. That would >> mean it was still possible to block

Re: Want to remove the archive

2020-05-11 Thread Satya Kotni
> Hi, > Please help me in removing this from the below archive > > https://www.mail-archive.com/dev@kafka.apache.org/msg104541.html > > Best Regards > Satya Kotni >

Re: [DISCUSS] KIP-608: Add a new method to AuthorizerServerInfo Interface

2020-05-11 Thread David Jacot
Hey, Thanks for the KIP. I think that having the possibilities to expose metrics in plugins such as the authorizer in a nice improvement. I do wonder if we could come up with a more generic way to do this that would apply to all plugins instead of having something specific for the authorized. Fo

Re: [VOTE] KIP-605 - Expand Connect Worker Internal Topic Settings

2020-05-11 Thread Konstantine Karantasis
This KIP falls in the category of necessary and straightforward KIPs. Thanks for the nice write-up Randall. +1 (binding) Konstantine On Mon, May 11, 2020 at 8:00 AM Randall Hauch wrote: > Ping for reviewers. > > I guess I never voted, so +1 (binding). > > On Thu, May 7, 2020 at 4:13 PM Christo

Re: [VOTE] KIP-586: Deprecate commit records without record metadata

2020-05-11 Thread Konstantine Karantasis
The proposal makes sense. Thanks for working on this Mario. +1 (binding) Konstantine On Wed, May 6, 2020 at 1:59 PM Randall Hauch wrote: > Thanks for putting this KIP together, Mario. > > +1 (binding) > > Randall > > On Mon, Apr 27, 2020 at 2:05 PM Mario Molina wrote: > > > Hi all, > > > > I'

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

2020-05-11 Thread Bruno Cadonna
Hi Matthias, Consuming a structured `TopologyDescription` object is not always the easiest option. If you want to use a topology description outside Kafka Streams, for instance to visualize it like with https://zz85.github.io/kafka-streams-viz/, a future-proof structured serialization format comes

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] MINOR: Use `forEach` and `ifPresent` to simplify Scala code (#8642) -- [...truncated 3.08 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [DISCUSS] KIP-608: Add a new method to AuthorizerServerInfo Interface

2020-05-11 Thread Jeff Huang
Hi David, I had thought about this before. I changed mind due to we want the metrics in Authorizer in 2.6 and time restraint. But I will reconsider this. Thanks for this suggestion. Jeff. On 2020/05/11 15:16:30, David Jacot wrote: > Hey, > > Thanks for the KIP. > > I think that having th

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] MINOR: Use `forEach` and `ifPresent` to simplify Scala code (#8642) -- [...truncated 3.08 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

[DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-11 Thread Anna Povzner
Hi, I just created KIP-612 to allow limiting connection creation rate on brokers, and would like to start a discussion. https://cwiki.apache.org/confluence/display/KAFKA/KIP-612%3A+Ability+to+Limit+Connection+Creation+Rate+on+Brokers Feedback and suggestions are welcome! Thanks, Anna

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-11 Thread Aakash Shah
I wasn't previously added to the dev mailing list, so I'd like to post my discussion with Andrew Schofield below for visibility and further discussion: Hi Andrew, Thanks for the reply. The main concern with this approach would be its backward compatibility. I’ve highlighted the thoughts around th

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] MINOR: Use `forEach` and `ifPresent` to simplify Scala code (#8642) -- [...truncated 6.12 MB...] org.apache.kafka.streams.test.ConsumerRecordFactor

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

2020-05-11 Thread Andy Coates
Hi Guozhang, Thanks for writing this up. I’m very interested to see this, so I hope you don’t mind me commenting. I’ve only really one comment to make, and that’s on the text printed for the serde classes: As I understand it, the name will either come from the passed in config, or may default

Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-11 Thread Sophie Blee-Goldman
Hey Bruno, Thanks for the KIP! I have one high-level concern, which is that we should consider reporting these metrics on the per-store level rather than instance-wide. I know I was the one who first proposed making it instance-wide, so bear with me: While I would still argue that the instance-wi

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-11 Thread Aakash Shah
Hi Chris, Thanks for the feedback! 1. Great point, this is the more correct general aim of the proposal. 2. Thanks for the suggestions on points a and b, they are both great. I will incorporate them. 3. Yep, I'll add this to the sample code and add an explanation. 4. Great point about the addi

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Guozhang Wang
I'm +1 on deprecating the auto.topic.creation in 2.6 as well. Guozhang On Mon, May 11, 2020 at 1:20 AM David Jacot wrote: > Hi all, > > First, I agree with what has been discussed. Having 3.x as the bridge > releases and entirely removing ZK in 4.0 makes total sense. > > Second, what would you

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Colin McCabe
On Mon, May 11, 2020, at 01:19, David Jacot wrote: > Hi all, > > First, I agree with what has been discussed. Having 3.x as the bridge > releases and entirely removing ZK in 4.0 makes total sense. > > Second, what would you think about removing the auto topics creation > in 3.0? It is not recomme

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Ismael Juma
Before we deprecate server side auto topic creation, we should have client side auto topic creation for the producer: https://cwiki.apache.org/confluence/display/KAFKA/KIP-487%3A+Client-side+Automatic+Topic+Creation+on+Producer Ismael On Mon, May 11, 2020 at 1:41 PM Colin McCabe wrote: > On Mo

Kafka Contributor List Addition

2020-05-11 Thread Saher Ahwal
Hi I kindly ask to be added to the Kafka contributor list. I was hoping to start contributing to the community. My company is relying on Kafka heavily and I would like to deeply understand the working of Kafka and contribute to the OS community as much as possible. My github handle is *saherahwa

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Andrew Otto
> Before we deprecate server side auto topic creation, we should have client side auto topic creation for the producer: Deprecating sounds fine, but before disabling it, it might be worthwhile to wait long enough for non Java clients to catch up to this too. :) On Mon, May 11, 2020 at 4:45 PM Is

Re: [DISCUSS] Kafka 3.0

2020-05-11 Thread Colin McCabe
Hi Michael, It would be better to discuss the background behind KIP-500 in a separate thread, since this thread is about the Kafka 3.0 release. As others have said, your questions are answered in the KIP. For example, "what is the actual goal?" is addressed in the motivation section. I agree

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9972: Only commit tasks with valid states (#8632) -- [...truncated 3.08 MB...] org.apache.kafka.streams.TopologyTestDriverTest > shouldProce

Re: Kafka Contributor List Addition

2020-05-11 Thread Israel Ekpo
Hi Saher, You can start by participating in discussions, testing release candidates, reviewing issues and submitting pull requests for documentation, code patches You can start by creating an account on the JIRA and you can be granted access to assign issues to yourself. https://issues.apache.or

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9972: Only commit tasks with valid states (#8632) -- [...truncated 3.08 MB...] org.apache.kafka.streams.TopologyTestDriverTest > shouldPunctu

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-11 Thread Andrew Schofield
Hi Aakash, Thanks for sorting out the replies to the mailing list. First, I do like the idea of improving error reporting in sink connectors. I'd like a simple way to put bad records onto the DLQ. I think this KIP is considerably more complicated than it seems. The guidance on the SinkTask.put(

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9972: Only commit tasks with valid states (#8632) -- [...truncated 3.06 MB...] org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shou

Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-11 Thread Guozhang Wang
Hello Sophie / Bruno, I've also thought about the leveling question, and one motivation I had for setting it in instance-level is that we want to expose it in INFO level: today our report leveling is not very finer grained --- which I think is sth. worth itself --- such that one have to either tur

Re: Kafka Contributor List Addition

2020-05-11 Thread Saher Ahwal
Thanks Isreal for the information. I created JIRA account with username *saherahwal* On Mon, May 11, 2020 at 2:33 PM Israel Ekpo wrote: > Hi Saher, > > You can start by participating in discussions, testing release candidates, > reviewing issues and submitting pull requests for documentation, co

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9956: Authorizer APIs may be invoked more than once for a given -- [...truncated 3.08 MB...] org.apache.kafka.streams.TopologyTestDriverTest

[jira] [Resolved] (KAFKA-9972) Corrupted standby task could be committed

2020-05-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9972?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-9972. Fix Version/s: 2.6.0 Resolution: Fixed > Corrupted standby task could be committed >

Re: Kafka Contributor List Addition

2020-05-11 Thread Matthias J. Sax
Saher, I added you to the list of contributors on Jira. You can now self-assign tickets. A good way to get started is to checkout the web page: https://kafka.apache.org/contributing Welcome to the Kafka community! -Matthias On 5/11/20 3:40 PM, Saher Ahwal wrote: > Thanks Isreal for the inform

Re: Want to remove the archive

2020-05-11 Thread Matthias J. Sax
I don't think it's possible to remove anything from the archive. Sorry. -Matthias On 5/11/20 2:54 AM, Satya Kotni wrote: >> Hi, >> Please help me in removing this from the below archive >> >> https://www.mail-archive.com/dev@kafka.apache.org/msg104541.html >> >> Best Regards >> Satya Kotni >> >

[jira] [Resolved] (KAFKA-9949) Flaky Test GlobalKTableIntegrationTest#shouldKStreamGlobalKTableLeftJoin

2020-05-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-9949. Fix Version/s: 2.6.0 Resolution: Fixed > Flaky Test GlobalKTableIntegrationTest#shou

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9956: Authorizer APIs may be invoked more than once for a given -- [...truncated 3.06 MB...] org.apache.kafka.streams.test.ConsumerRecordFacto

Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-11 Thread John Roesler
Thanks, all. If you don’t mind, I’ll pitch in a few cents’ worth. In my life I’ve generally found more granular metrics to be more useful, as long as there’s a sane way to roll them up. It does seem nice to see it on the per-store level. For roll-up purposes, the task and thread tags should be

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9956: Authorizer APIs may be invoked more than once for a given -- [...truncated 6.16 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9921: explicit handling of null values with retainDuplicates [github] KAFKA-9949: Fix flaky GlobalKTableIntegrationTest (#8635) -- [...trunca

Re: [VOTE] KIP-221: Enhance KStream with Connecting Topic Creation and Repartition Hint

2020-05-11 Thread Matthias J. Sax
As an afterthought to KIP-221, I am wondering if we should deprecate `KStream#through()`? The reasoning is that I assume that most people don't want to manage topics manually anyway and thus it might be good to guide users to use repartition(). Furthermore, through() is really just syntactic sugar

[jira] [Created] (KAFKA-9979) The "--zookeeper" option can't be used with bin/kafka-configs.sh . The document should be updated

2020-05-11 Thread jiamei xie (Jira)
jiamei xie created KAFKA-9979: - Summary: The "--zookeeper" option can't be used with bin/kafka-configs.sh . The document should be updated Key: KAFKA-9979 URL: https://issues.apache.org/jira/browse/KAFKA-9979

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9921: explicit handling of null values with retainDuplicates [github] KAFKA-9949: Fix flaky GlobalKTableIntegrationTest (#8635) -- [...truncat

Build failed in Jenkins: kafka-2.5-jdk8 #113

2020-05-11 Thread Apache Jenkins Server
See Changes: [matthias] KAFKA-9921: explicit handling of null values with retainDuplicates -- [...truncated 2.92 MB...] org.apache.kafka.streams.MockProcessorContextTest > s

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9921: explicit handling of null values with retainDuplicates [github] KAFKA-9949: Fix flaky GlobalKTableIntegrationTest (#8635) -- [...trunc

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

2020-05-11 Thread Apache Jenkins Server
See Changes: -- Started by an SCM change Started by an SCM change Running as SYSTEM [EnvInject] - Loading node environment variables. Building remotely on H30 (ubuntu) in workspace

[jira] [Resolved] (KAFKA-9979) The "--zookeeper" option can't be used with bin/kafka-configs.sh . The document should be updated

2020-05-11 Thread jiamei xie (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9979?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jiamei xie resolved KAFKA-9979. --- Resolution: Duplicate It has been fixed > The "--zookeeper" option can't be used with bin/kafka-conf

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

2020-05-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9466: Update Kafka Streams docs for KIP-447 (#8621) -- [...truncated 3.08 MB...] org.apache.kafka.streams.test.OutputVerifierTest > shouldFail

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

2020-05-11 Thread Apache Jenkins Server
See

[DISCUSS] KIP-614: Add Prefix Scan support for State Stores

2020-05-11 Thread Sagar
Hi All, I would like to start a discussion on the KIP that I created below to add prefix scan support in State Stores: https://cwiki.apache.org/confluence/display/KAFKA/KIP-614%3A+Add+Prefix+Scan+support+for+State+Stores Thanks! Sagar.