Re: [DISCUSS] KIP-621: Deprecate and replace DescribeLogDirsResult.all() and .values()

2020-07-10 Thread Colin McCabe
Yeah. The issue with subclassing is that it's a source compatibility break, although not (I think) a binary compatibility break. I don't think it's really worth it given that it leaves us with a weird class hierarchy, and we still need to do a hard compatibility break later to fix the real

Re: [DISCUSS] KIP-621: Deprecate and replace DescribeLogDirsResult.all() and .values()

2020-07-10 Thread Colin McCabe
Incidentally, whenever we do a hard compatibility break with this API, we should probably get rid of the function that iterates over every replica on the broker. That's not a scalable API for the future. We also probably should have an API to list just the directories that are moving, which

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Yuriy Badalyantc
Hi, Matthias, It's not directly mentioned in the KIP, but I added all missing Java serdes. I mentioned it in the pull request description: https://github.com/apache/kafka/pull/8955 And also, this KIP originally was based on a pull request where I added missing java serdes :)

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-07-10 Thread Colin McCabe
Hi Unmesh, Yes, once the last stable offset advanced, we would consider the topic creation to be done, and then we could return success to the client. best, Colin On Thu, Jul 9, 2020, at 19:44, Unmesh Joshi wrote: > It still needs HighWaterMark / LastStableOffset to be advanced by two >

Re: [DISCUSS] KIP-431: Support of printing additional ConsumerRecord fields in DefaultMessageFormatter

2020-07-10 Thread Badai Aqrandista
Hu I would make it more readable and move the key next to the value like this: CreateTime:1592475472398|Partition:3|Offset:0|Headers:h1=v1,h2=v2|key1|value1 What do you think? In future KIP, we'll add some formatting like David Jacot suggested. Regards Badai On Fri, Jul 10, 2020 at 2:57 PM

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-07-10 Thread Unmesh Joshi
I was thinking that we might need something like multi-operation record in zookeeper to atomically create topic and partition records when this multi record is committed. This way metadata will have both the TopicRecord and PartitionRecord

Re: [VOTE] KIP-621: Deprecate and replace DescribeLogDirsResult.all() and .values()

2020-07-10 Thread Dongjin Lee
+1. (non-binding) As of present, it has 3 bindings (Colin, Manikumar, and Mickael) and 2 non-bindings (David, Dongjin). Since it has 3 binding +1 votes and more binding +1 votes than -1 votes, this KIP is accepted. Thanks everyone for the votes. On Thu, Jul 9, 2020 at 6:19 PM Mickael Maison

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

2020-07-10 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10249: don't try to read un-checkpointed offsets of in-memory -- [...truncated 1.91 MB...] kafka.api.GroupAuthorizerIntegrationTest >

[jira] [Resolved] (KAFKA-10265) FetchRequest and FetchResponse should use the generated message classes

2020-07-10 Thread David Arthur (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10265?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Arthur resolved KAFKA-10265. -- Resolution: Duplicate > FetchRequest and FetchResponse should use the generated message

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread John Roesler
Ah, thanks Yuriy, Sorry if this wasn't clear, but _all_ public API changes have to be explicitly included in the KIP. Can you just enumerate all the contents of the new API? Thanks, John On Fri, Jul 10, 2020, at 04:54, Yuriy Badalyantc wrote: > Hi, Matthias, > > It's not directly mentioned in

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Yuriy Badalyantc
Oh, ok. I have done that. Just didn't know that it was necessary. -Yuriy On Fri, Jul 10, 2020 at 9:30 PM John Roesler wrote: > Ah, thanks Yuriy, > > Sorry if this wasn't clear, but _all_ public API changes have to > be explicitly included in the KIP. Can you just enumerate all > the contents

Re: [VOTE] KIP-554: Add Broker-side SCRAM Config API

2020-07-10 Thread Rajini Sivaram
+1 (binding) Thanks for the KIP, Colin! Regards, Rajini On Thu, Jul 9, 2020 at 8:49 PM Colin McCabe wrote: > Hi all, > > I'd like to call a vote for KIP-554: Add a broker-side SCRAM configuration > API. The KIP is here: https://cwiki.apache.org/confluence/x/ihERCQ > > The previous

Re: [DISCUSS] KIP-621: Deprecate and replace DescribeLogDirsResult.all() and .values()

2020-07-10 Thread Dongjin Lee
Hi Tom and Colin, I see. Thanks for the comprehensive explanation. I learned a lot. Although my approach includes a new member field `DescribeLogDirsResult.LogDirInfo.tpToReplicaInfos` to keep binary compatibility, but anyway, you are right - it does not worth except small consistency gain.

[jira] [Created] (KAFKA-10265) FetchRequest and FetchResponse should use the generated message classes

2020-07-10 Thread David Arthur (Jira)
David Arthur created KAFKA-10265: Summary: FetchRequest and FetchResponse should use the generated message classes Key: KAFKA-10265 URL: https://issues.apache.org/jira/browse/KAFKA-10265 Project:

Re: [VOTE] KIP-621: Deprecate and replace DescribeLogDirsResult.all() and .values()

2020-07-10 Thread Tom Bentley
Thanks Dongjin, I was hoping to close the vote, but was waiting till we had consensus around the points you raised on the discussion thread. On Fri, Jul 10, 2020 at 2:39 PM Dongjin Lee wrote: > +1. (non-binding) > > As of present, it has 3 bindings (Colin, Manikumar, and Mickael) and 2 >

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-10 Thread Boyang Chen
Thanks Leah and Sophie for the KIP. 1. I'm a bit surprised that we don't have an advance time. Could we elaborate how the storage layer is structured? 2. IIUC, there will be extra cost in terms of fetching aggregation results, since we couldn't pre-aggregate until the user asks for it. Would be

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-10 Thread Sophie Blee-Goldman
Thanks Leah! This kind of assumes an implicit answer to Matthias's question, but I was wondering if we should take this opportunity to choose a better default value for the grace period. Note that the default of -1 in the TimeWindows class, for example, ultimately gets translated into a default

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-10 Thread Leah Thomas
Hey Matthias, Thanks for pointing that out. I added the following to the Propose Changes section of the KIP: "Records that come out of order will be processed the same way as in-order records, as long as they fall within the grace period. Any new windows created by the late record will still be

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Matthias J. Sax
Thanks Yuriy! What about `VoidSerde` ? It's not listed. It might also be nice to add a short sentence and state that in addition to fixing the name collisions, the KIP will also close the gap of out-of-the-box serdes and add missing Serdes that are offered in Java to Scala. -Matthias On

Re: [VOTE] KIP-554: Add Broker-side SCRAM Config API

2020-07-10 Thread Boyang Chen
Hey Colin, thanks for the KIP. One question I have about AlterScramUsers RPC is whether we could consolidate the deletion list and alteration list, since in response we only have a single list of results. The further benefit is to reduce unintentional duplicate entries for both deletion and

[GitHub] [kafka-site] vvcephei merged pull request #274: MINOR: Add vvcephei to KEYS

2020-07-10 Thread GitBox
vvcephei merged pull request #274: URL: https://github.com/apache/kafka-site/pull/274 This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go

Re: [VOTE] KIP-620 Deprecate ConsumerConfig#addDeserializerToConfig(Properties, Deserializer, Deserializer) and ProducerConfig#addSerializerToConfig(Properties, Serializer, Serializer)

2020-07-10 Thread Chia-Ping Tsai
> I don't think my question gets answered, Sorry for incorrect response :( > why would deprecating the map > based `addSerializerToConfig` break user's recompilation? If you worry > about warnings, we could refactor out the content and create a > package-private `attachSerializersToConfig` or

Build failed in Jenkins: kafka-2.6-jdk8 #76

2020-07-10 Thread Apache Jenkins Server
See Changes: [vvcephei] KAFKA-10191 fix flaky StreamsOptimizedTest (#8913) -- [...truncated 3.15 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-10 Thread Colin McCabe
Hi all, Thanks for the KIP. I took a look and one thing that stood out to me is that the more metadata we have, the more storage we will need on local disk for the rocksDB database. This seems like it contradicts some of the goals of the project. Ideally the space we need on local disk

[GitHub] [kafka-site] vvcephei opened a new pull request #274: MINOR: Add vvcephei to KEYS

2020-07-10 Thread GitBox
vvcephei opened a new pull request #274: URL: https://github.com/apache/kafka-site/pull/274 This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above

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

2020-07-10 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10263: Do not assign standby for revoking stateless tasks (#9005) -- [...truncated 3.19 MB...]

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-07-10 Thread Boyang Chen
Thanks for the update! On Fri, Jul 3, 2020 at 3:18 PM Joel Wee wrote: > Thanks all for voting! > > I am closing the vote as accepted with three binding +1 votes (Boyang, > Guozhang, John). > > Thanks John for the suggestion. I think that makes sense. I have updated > the KIP to say that only

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-07-10 Thread Guozhang Wang
Hello Colin, Thanks for the nice written KIP. A few meta comments: 1) We need to talk a bit about broker failure detection: is that piggy backed with fencing? i.e. should the controller immediately migrate leader partitions from the fenced brokers? On one side, when a broker is fenced it cannot

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

2020-07-10 Thread Apache Jenkins Server
See

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

2020-07-10 Thread Apache Jenkins Server
See

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-07-10 Thread Matthias J. Sax
Randall, we found another blocker: https://issues.apache.org/jira/browse/KAFKA-10262 Luckily, we have already a PR for it. -Matthias On 7/8/20 3:05 PM, Sophie Blee-Goldman wrote: > Hey Randall, > > We just discovered another regression in 2.6: >

Build failed in Jenkins: kafka-2.6-jdk8 #77

2020-07-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-10263: Do not assign standby for revoking stateless tasks (#9005) -- [...truncated 3.15 MB...]

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Yuriy Badalyantc
Ok, I mentioned adding missing serdes in the Proposed Change paragraph. About VoidSerde. I didn't add it intentionally. The semantic of the Unit (scala's void) type is not clear in terms of the data. If kafka topic contains messages of type Unit, what does it actually means? That there is always

[jira] [Created] (KAFKA-10267) [Documentation] | Correction in kafka-console-producer command

2020-07-10 Thread Hemant Girase (Jira)
Hemant Girase created KAFKA-10267: - Summary: [Documentation] | Correction in kafka-console-producer command Key: KAFKA-10267 URL: https://issues.apache.org/jira/browse/KAFKA-10267 Project: Kafka

[jira] [Created] (KAFKA-10266) Fix connector configs in docs to mention the correct default value inherited from worker configs

2020-07-10 Thread Konstantine Karantasis (Jira)
Konstantine Karantasis created KAFKA-10266: -- Summary: Fix connector configs in docs to mention the correct default value inherited from worker configs Key: KAFKA-10266 URL:

Re: [VOTE] KIP-620 Deprecate ConsumerConfig#addDeserializerToConfig(Properties, Deserializer, Deserializer) and ProducerConfig#addSerializerToConfig(Properties, Serializer, Serializer)

2020-07-10 Thread Boyang Chen
Thanks for the update. One nit for the KIP is to format the signature indentation for all code templates, like: public static Properties addSerializerToConfig(Properties properties, Serializer keySerializer, Serializer valueSerializer) Other than that, +1 (binding) from me. On Fri, Jul 10,

Re: [DISCUSS] KIP-431: Support of printing additional ConsumerRecord fields in DefaultMessageFormatter

2020-07-10 Thread Badai Aqrandista
Hi all The vote for KIP-431 has passed with 3 binding and 1 non-binding +1s, and no objections. Thanks everyone for reviews and feedback. Regards, Badai On Fri, Jul 10, 2020 at 5:22 PM Badai Aqrandista wrote: > > Hu > > I would make it more readable and move the key next to the value like