Re: [DISCUSS] KIP-637 Include min.insync.replicas in MetadataResponse to make Producer smarter in partitioning events

2020-07-07 Thread Arvin Zheng
Thanks for the comment. Yes, it's indeed a downside that this will increase the size of the metadata response, I was thinking if it's worth providing this information to the Producer conditionally, e.g. add a config to Producer to allow people choose whether to include this information in the

[jira] [Resolved] (KAFKA-10221) Backport fix for KAFKA-9603 to 2.5

2020-07-07 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler resolved KAFKA-10221. -- Resolution: Fixed > Backport fix for KAFKA-9603 to 2.5 > ---

[jira] [Created] (KAFKA-10246) AbstractProcessorContext topic() throws NullPointerException when modifying a state store within the DSL from a punctuator

2020-07-07 Thread Peter Pringle (Jira)
Peter Pringle created KAFKA-10246: - Summary: AbstractProcessorContext topic() throws NullPointerException when modifying a state store within the DSL from a punctuator Key: KAFKA-10246 URL:

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

2020-07-07 Thread John Roesler
Hi Badai, Thanks for picking this up. I've reviewed the KIP document and the threads you linked. I think we may want to make more improvements in the future to the printing of headers in particular, but this KIP seems like a clear benefit already. I think we can take it incrementally. I'm +1

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

2020-07-07 Thread Ron Dagostino
HI Colin. Thanks for the KIP. Here is some feedback and various questions. "*Controller processes will listen on a separate port from brokers. This will be true even when the broker and controller are co-located in the same JVM*". I assume it is possible that the port numbers could be the same

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

2020-07-07 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10191 fix flaky StreamsOptimizedTest (#8913) -- [...truncated 6.38 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

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

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

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

2020-07-07 Thread Colin McCabe
Hi all, I posted a KIP about how the quorum-based controller envisioned in KIP-500 will work. Please take a look here: https://cwiki.apache.org/confluence/x/4RV4CQ best, Colin

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

2020-07-07 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-07-07 Thread William Bottrell
Sure, I would appreciate help from Piotr creating an example. On Tue, Jul 7, 2020 at 12:03 PM Boyang Chen wrote: > Hey John, > > since ProcessorContext is a public API, I couldn't be sure that people > won't try to extend it. Without a default implementation, user code > compilation will break.

Jenkins build is back to normal : kafka-trunk-jdk14 #274

2020-07-07 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-07-07 Thread Boyang Chen
Hey John, since ProcessorContext is a public API, I couldn't be sure that people won't try to extend it. Without a default implementation, user code compilation will break. William and Piotr, it seems that we haven't added any example usage of the new API, could we try to address that? It should

[VOTE] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-07-07 Thread William Bottrell
Hi everyone, I'd like to start a vote for adding two new time API's to ProcessorContext. Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext Thanks

[jira] [Resolved] (KAFKA-10222) Incorrect methods show up in 0.10 Kafka Streams docs

2020-07-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-10222. - Resolution: Fixed > Incorrect methods show up in 0.10 Kafka Streams docs >

[jira] [Resolved] (KAFKA-10191) fix flaky StreamsOptimizedTest

2020-07-07 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-10191. - Resolution: Fixed > fix flaky StreamsOptimizedTest >

[jira] [Created] (KAFKA-10245) Using vulnerable log4j version

2020-07-07 Thread Pavel Kuznetsov (Jira)
Pavel Kuznetsov created KAFKA-10245: --- Summary: Using vulnerable log4j version Key: KAFKA-10245 URL: https://issues.apache.org/jira/browse/KAFKA-10245 Project: Kafka Issue Type: Bug

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

2020-07-07 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10243; ConcurrentModificationException while processing connection -- [...truncated 3.16 MB...]

[GitHub] [kafka-site] mjsax commented on pull request #271: MINOR: Fix table of contents in protocol page

2020-07-07 Thread GitBox
mjsax commented on pull request #271: URL: https://github.com/apache/kafka-site/pull/271#issuecomment-655024302 Thanks @wkodate This is an automated message from the Apache Git Service. To respond to the message, please log

[GitHub] [kafka-site] mjsax merged pull request #271: MINOR: Fix table of contents in protocol page

2020-07-07 Thread GitBox
mjsax merged pull request #271: URL: https://github.com/apache/kafka-site/pull/271 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 to

[GitHub] [kafka-site] mjsax commented on pull request #272: KAFKA-10222:Incorrect methods show up in 0.10 Kafka Streams docs

2020-07-07 Thread GitBox
mjsax commented on pull request #272: URL: https://github.com/apache/kafka-site/pull/272#issuecomment-655021969 We can still merge this PR, because otherwise the web-page would not be updated. This is an automated message

[GitHub] [kafka-site] mjsax merged pull request #272: KAFKA-10222:Incorrect methods show up in 0.10 Kafka Streams docs

2020-07-07 Thread GitBox
mjsax merged pull request #272: URL: https://github.com/apache/kafka-site/pull/272 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 to

Re: [DISCUSSION] KIP-619: Add internal topic creation support

2020-07-07 Thread Cheng Tan
Hi Colin, Thanks for the comments. I’ve modified the KIP accordingly. > I think we need to understand which of these limitations we will carry > forward and which we will not. We also have the option of putting > limitations just on consumer offsets, but not on other internal topics. In

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

2020-07-07 Thread Jun Rao
Hi, Ying, Thanks for the update. It's good to see the progress on this. Please let us know when you are done updating the KIP wiki. Jun On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng wrote: > Hi Jun, > > Satish and I have added more design details in the KIP, including how to > keep consistency

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

2020-07-07 Thread Ying Zheng
Hi Jun, Satish and I have added more design details in the KIP, including how to keep consistency between replicas (especially when there is leadership changes / log truncations) and new metrics. We also made some other minor changes in the doc. We will finish the KIP changes in the next couple

Re: [VOTE] KIP-632: Add DirectoryConfigProvider

2020-07-07 Thread David Jacot
+1 (non-binding). Thanks for the KIP! On Tue, Jul 7, 2020 at 12:54 PM Tom Bentley wrote: > Hi, > > I'd like to start a vote on KIP-632, which is about making the config > provider mechanism more ergonomic on Kubernetes: > > >

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

2020-07-07 Thread Jun Rao
Hi, Satish, Harsha, Any new updates on the KIP? This feature is one of the most important and most requested features in Apache Kafka right now. It would be helpful if we can make sustained progress on this. Could you share how far along is the design/implementation right now? Is there anything

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

2020-07-07 Thread Boyang Chen
Ok, after a second thought, keeping a function which still has production reference is ok. We probably should not make it public in the first place, but this is not high priority either. On Tue, Jul 7, 2020 at 9:03 AM Chia-Ping Tsai wrote: > > do we just suggest they no longer have any

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

2020-07-07 Thread Chia-Ping Tsai
> do we just suggest they no longer have any production use case? yep > KafkaProducer internal only. Do we also want to deprecate this public API as > well? We have to make sure users' code can keep working beyond recompilation when migrating to "next" release. Hence, deprecation cycle is

[DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'

2020-07-07 Thread Chia-Ping Tsai
hi all, I would like to start the discussion for KIP-641. https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 Many thanks, Chia-Ping

[jira] [Resolved] (KAFKA-10243) ConcurrentModificationException while processing connection setup timeouts

2020-07-07 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10243?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-10243. Reviewer: Rajini Sivaram Resolution: Fixed > ConcurrentModificationException while

[jira] [Created] (KAFKA-10244) An new java interface to replace 'kafka.common.MessageReader'

2020-07-07 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-10244: -- Summary: An new java interface to replace 'kafka.common.MessageReader' Key: KAFKA-10244 URL: https://issues.apache.org/jira/browse/KAFKA-10244 Project: Kafka

Re: [DISCUSS] KIP-363

2020-07-07 Thread Colin McCabe
Hi Tom, Thanks for this. I think the tough part is probably the few messages that are still using manual serialization, which can't be easily converted to using this. So we will probably have to upgrade them to using automatic generation, or accept a little inconsistency for a while until

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

2020-07-07 Thread Badai Aqrandista
Hi all After resurrecting the discussion thread [1] for KIP-431 and have not received any further feedback for 2 weeks, I would like to resurrect the voting thread [2] for KIP-431. I have updated KIP-431 wiki page

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

2020-07-07 Thread Colin McCabe
Thanks, Tom. I tried to think of a better way to do this, but I think you're right that we probably just need different methods. +1 (binding). best, Colin On Mon, Jul 6, 2020, at 01:14, Tom Bentley wrote: > Hi, > > I'd like to start a vote on KIP-621 which is about deprecating methods in >

Re: [DISCUSS] Include min.insync.replicas in MetadataResponse to make Producer smarter in partitioning events

2020-07-07 Thread Colin McCabe
Hi Arvin, Thanks for the KIP. Unfortunately, I don't think this makes sense since it would increase the amount of data we send back in the metadata response, which is pretty bad for scalability. In general we probably want to avoid the case where we don't have the appropriate number of

Re: [DISCUSS] KIP-638: Deprecate DescribeLogDirsResponse.[LogDirInfo, ReplicaInfo]

2020-07-07 Thread Mickael Maison
Hi Dongjin, It looks like this KIP is addressing the same issue as KIP-621: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158862109 On Tue, Jul 7, 2020 at 2:29 PM Dongjin Lee wrote: > > Hi devs, > > I hope to start the discussion of KIP-638, which aims to fix a glitch in >

Re: Untrimmed Index Files resulting in data loss

2020-07-07 Thread Ismael Juma
Hi John, Thanks for reporting the issue. Let's continue the discussion in the PR. Ismael On Mon, Jul 6, 2020 at 7:13 PM John Malizia wrote: > Hi there, about a week ago I submitted an issue report and an associated PR > > https://issues.apache.org/jira/browse/KAFKA-10207 >

[DISCUSS] KIP-638: Deprecate DescribeLogDirsResponse.[LogDirInfo, ReplicaInfo]

2020-07-07 Thread Dongjin Lee
Hi devs, I hope to start the discussion of KIP-638, which aims to fix a glitch in Admin#describeLogDirs method. - KIP: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866169 - Jira: https://issues.apache.org/jira/browse/KAFKA-8794 All kinds of feedback will be greatly

Re: Feedback: Print schemaId using bin/kafka-dump-log.sh

2020-07-07 Thread Adam Bellemare
Hi Mohanraj While I see the usefulness of your suggestion, the main issue is that you're using the Confluent schema registry's conventions and hardwiring them into Kafka core. Given that Confluent's standards are not part of Kafka's official standards, I do not think you will get approval to

[VOTE] KIP-632: Add DirectoryConfigProvider

2020-07-07 Thread Tom Bentley
Hi, I'd like to start a vote on KIP-632, which is about making the config provider mechanism more ergonomic on Kubernetes: https://cwiki.apache.org/confluence/display/KAFKA/KIP-632%3A+Add+DirectoryConfigProvider Please take a look if you have time. Many thanks, Tom

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

2020-07-07 Thread Apache Jenkins Server
See Changes: [boyang] KAFKA-10239: Make GroupInstanceId ignorable in DescribeGroups (#8989) -- [...truncated 5.92 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

[jira] [Created] (KAFKA-10243) ConcurrentModificationException while processing connection setup timeouts

2020-07-07 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-10243: -- Summary: ConcurrentModificationException while processing connection setup timeouts Key: KAFKA-10243 URL: https://issues.apache.org/jira/browse/KAFKA-10243

NotEnoughReplicasException: The size of the current ISR Set(2) is insufficient to satisfy the min.isr requirement of 3

2020-07-07 Thread Nag Y
I had the following setup Brokers : 3 - all are up and running with min.insync.replicas=3. I created a topic with the following configuration bin\windows\kafka-topics --zookeeper 127.0.0.1:2181 --topic topic-ack-all --create --partitions 4 --replication-factor 3 I triggered the producer with

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

2020-07-07 Thread Apache Jenkins Server
See

Re: KIP-560 Discuss

2020-07-07 Thread Boyang Chen
No worry! Could you address Matthias' comments in this mailing thread? Seems we still have some gaps. On Mon, Jul 6, 2020 at 7:14 AM Sang wn Lee wrote: > I'm sorry. > I just modified the KIP! > > On 2020/03/07 20:09:47, "Matthias J. Sax" wrote: > > -BEGIN PGP SIGNED MESSAGE- > > Hash:

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

2020-07-07 Thread Boyang Chen
Thanks for the KIP. One question I have is that when we refer to the two methods as useless, do we just suggest they no longer have any production use case? If this is the case, Producer#addSerializerToConfig(Map configs, keySerializer, valueSerializer) is only used in KafkaProducer internal only.

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

2020-07-07 Thread Yuriy Badalyantc
So, what's next? It's my first KIP and I'm not familiar with all processes. -Yuriy On Mon, Jul 6, 2020 at 1:32 AM John Roesler wrote: > Hi Yuriy, > > Thanks for the update! It looks good to me. > > Thanks, > John > > On Sun, Jul 5, 2020, at 03:27, Yuriy Badalyantc wrote: > > Hi John. > > > > I

[DISCUSS] KIP-637 Include min.insync.replicas in MetadataResponse to make Producer smarter in partitioning events

2020-07-07 Thread Arvin Zheng
Updated the subject to add KIP number Arvin Zheng 于2020年7月6日周一 上午10:38写道: > Hi everyone, > > I would like to start the discussion for KIP-637 > >

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

2020-07-07 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10166: checkpoint recycled standbys and ignore empty rocksdb base [github] MINOR: document timestamped state stores (#8920) [github] KAFKA-9930: Adjust ReplicaFetcherThread logging

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

2020-07-07 Thread Manikumar
+1 (binding) Thanks for the KIP. On Wed, Jun 10, 2020 at 11:43 PM Matthias J. Sax wrote: > Yes, it does. > > I guess many people are busy wrapping up 2.6 release. Today is code freeze. > > > -Matthias > > > On 6/10/20 12:11 AM, Chia-Ping Tsai wrote: > > hi Matthias, > > > > Does this

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

2020-07-07 Thread Apache Jenkins Server
See Changes: [github] MINOR: document timestamped state stores (#8920) [github] KAFKA-9930: Adjust ReplicaFetcherThread logging when processing [github] KAFKA-10239: Make GroupInstanceId ignorable in