Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

2020-07-27 Thread Boyang Chen
On Mon, Jul 27, 2020 at 4:58 AM Unmesh Joshi wrote: > Just checked etcd and zookeeper code, and both support leader to step down > as a follower to make sure there are no two leaders if the leader has been > disconnected from the majority of the followers > For etcd this is

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

2020-07-27 Thread Boyang Chen
Hey there, I'm re-opening this thread because after some initial implementations started, we spotted some gaps in the approved KIP as well as some inconsistencies with KIP-631 controller. There are a couple of addendums to the existing KIP, specifically: 1. As the controller is foreseen to be

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Adjust 'release.py' script to use shell when using gradlewAll and [github] MINOR: Remove staticmethod tag to be able to use logger of instance [github] MINOR: remove

[jira] [Created] (KAFKA-10317) Global thread should honor shutdown signal during bootstrapping

2020-07-27 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-10317: --- Summary: Global thread should honor shutdown signal during bootstrapping Key: KAFKA-10317 URL: https://issues.apache.org/jira/browse/KAFKA-10317 Project: Kafka

Re: [DISCUSS] KIP-618: Atomic commit of source connector records and offsets

2020-07-27 Thread Ning Zhang
Hello Chris, That is an interesting KIP. I have a couple of questions: (1) in section of pseudo-code, what if the failure happens between 4(b) and 5(a), meaning after the producer commit the transaction, and before task.commitRecord(). (2) in section "source task life time", what is the

Re: [DISCUSS] KIP-647: Add ability to handle late messages in streams-aggregation

2020-07-27 Thread Matthias J. Sax
Thanks for the KIP Igor. What you propose sounds a little bit like a "dead-letter-queue" pattern. Thus, I am wondering if we should try to do a built-in "dead-letter-queue" feature that would be general purpose? For example, uses can drop message in the source node if they don't have a valid

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-27 Thread John Roesler
Thanks for the reply, Sophie. Yes, I'd neglected to specify that Windows will implement maxSize() by delegating to size(). It's updated now. I'd also neglected to say that I plan to alter both windowBy methods to use the new interface now. Because Windows will implement the new interface, all

Build failed in Jenkins: kafka-2.3-jdk8 #220

2020-07-27 Thread Apache Jenkins Server
See Changes: [matthias] MINOR: Remove staticmethod tag to be able to use logger of instance -- [...truncated 2.70 MB...] kafka.log.LogCleanerTest >

[jira] [Created] (KAFKA-10316) Consider renaming getter method for Interactive Queries

2020-07-27 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-10316: --- Summary: Consider renaming getter method for Interactive Queries Key: KAFKA-10316 URL: https://issues.apache.org/jira/browse/KAFKA-10316 Project: Kafka

Re: [VOTE] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-07-27 Thread Matthias J. Sax
+1 (binding) On 7/27/20 4:55 PM, Guozhang Wang wrote: > +1. Thanks Jorge for bringing in this KIP! > > Guozhang > > On Mon, Jul 27, 2020 at 10:07 AM Leah Thomas wrote: > >> Hi Jorge, >> >> Looks great. +1 (non-binding) >> >> Best, >> Leah >> >> On Thu, Jul 16, 2020 at 6:39 PM Sophie

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [matthias] MINOR: Remove staticmethod tag to be able to use logger of instance -- [...truncated 2.93 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Remove staticmethod tag to be able to use logger of instance [github] MINOR: remove NewTopic#NO_PARTITIONS and NewTopic#NO_REPLICATION_FACTOR

Re: [VOTE] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-07-27 Thread Guozhang Wang
+1. Thanks Jorge for bringing in this KIP! Guozhang On Mon, Jul 27, 2020 at 10:07 AM Leah Thomas wrote: > Hi Jorge, > > Looks great. +1 (non-binding) > > Best, > Leah > > On Thu, Jul 16, 2020 at 6:39 PM Sophie Blee-Goldman > wrote: > > > Hey Jorge, > > > > Thanks for the reminder -- +1

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Adjust 'release.py' script to use shell when using gradlewAll and -- [...truncated 5.03 MB...]

Re: [DISCUSS] KIP-645: Replace abstract class Windows with a proper interface

2020-07-27 Thread Sophie Blee-Goldman
Thanks for taking the time to really fill in the background details for this KIP. The Motivation section is very informative. Hopefully this will also serve as a warning against making similar such mistakes in the future :P I notice that the `Window` class that parametrizes

[jira] [Created] (KAFKA-10315) Consider to throw exception for failed fetch requests

2020-07-27 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-10315: --- Summary: Consider to throw exception for failed fetch requests Key: KAFKA-10315 URL: https://issues.apache.org/jira/browse/KAFKA-10315 Project: Kafka

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-27 Thread Leah Thomas
A small tweak - to make it more clear to users that grace is required, as well as cleaning up some of the confusing grammar semantics of windows, the main builder method for *slidingWindows* will be *withSizeAndGrace* instead of *of*. This looks like it'll be the last change (for now) on the

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update dependencies for Kafka 2.7 (part 1) (#9082) -- [...truncated 1.91 MB...] kafka.integration.MinIsrConfigTest > testDefaultKafkaConfig

Re: [VOTE] KIP-572: Improve timeouts and retires in Kafka Streams

2020-07-27 Thread Jason Gustafson
Hi Matthias, Sorry for jumping in so late here. I am trying to understand why it was necessary to deprecate `retries` in the producer. One of the use cases that I see in practice is setting `retries` to 0. This allows applications to control the retry semantics themselves. For example, I have

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Remove staticmethod tag to be able to use logger of instance [github] MINOR: remove NewTopic#NO_PARTITIONS and NewTopic#NO_REPLICATION_FACTOR

Re: [DISCUSS] KIP-630: Kafka Raft Snapshot

2020-07-27 Thread Ron Dagostino
Hi Jose. Thanks for the KIP. Here are some questions and some nit corrections. <<< In KIP-500 the Kafka Controller, which is the quorum leader from KIP-595, will materialize the entries in the metadata log into memory. Technically I think the quorum leader is referred to as the Active

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

2020-07-27 Thread Apache Jenkins Server
See

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

2020-07-27 Thread Guozhang Wang
+1 (binding). Thanks. On Mon, Jul 27, 2020 at 9:24 AM Bill Bejeck wrote: > Thanks for the KIP Bruno. > > +1 (binding) > > -Bill > > On Mon, Jul 27, 2020 at 4:26 AM Navinder Brar > wrote: > > > +1 (non-binding). Thanks for the KIP, Bruno. > > > > ~Navinder > > > > On Friday, 24 July, 2020,

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

2020-07-27 Thread Guozhang Wang
Thanks Bruno, That is clear. I think my misunderstanding is that I thought compaction is done by loading sst files into block cache (not OS), do the merge-sort and then write back to sst. But on a second thought I agree that it is not a good use case fitted for caching anyways. Using non-cached

Re: [VOTE] 2.6.0 RC1

2020-07-27 Thread Randall Hauch
Thanks, John. Looks like we're still trying to get a green build for https://github.com/apache/kafka/pull/9066. On Fri, Jul 24, 2020 at 3:46 PM John Roesler wrote: > Hi Randall, > > I'm sorry to say we have also identified that this flaky test > failure turned out to be a real blocker bug: >

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update dependencies for Kafka 2.7 (part 1) (#9082) [github] MINOR: Adjust 'release.py' script to use shell when using gradlewAll and --

Re: [VOTE] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-07-27 Thread Leah Thomas
Hi Jorge, Looks great. +1 (non-binding) Best, Leah On Thu, Jul 16, 2020 at 6:39 PM Sophie Blee-Goldman wrote: > Hey Jorge, > > Thanks for the reminder -- +1 (non-binding) > > Cheers, > Sophie > > On Thu, Jul 16, 2020 at 4:06 PM Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote:

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

2020-07-27 Thread Jun Rao
Hi, Colin, Thanks for the KIP. A few comments below. 10. Some of the choices in this KIP are not consistent with KIP-595. It would be useful to make consistent choices between the two KIPs. 10.1 KIP-595 doesn't use a separate Heartbeat request and heartbeat is piggybacked through the Fetch

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

2020-07-27 Thread Bill Bejeck
Thanks for the KIP Bruno. +1 (binding) -Bill On Mon, Jul 27, 2020 at 4:26 AM Navinder Brar wrote: > +1 (non-binding). Thanks for the KIP, Bruno. > > ~Navinder > > On Friday, 24 July, 2020, 08:41:03 pm IST, John Roesler < > vvcep...@apache.org> wrote: > > Thanks, Bruno! > > I'm +1

[jira] [Created] (KAFKA-10314) KafkaStorageException on reassignment when offline log directories exist

2020-07-27 Thread Noa Resare (Jira)
Noa Resare created KAFKA-10314: -- Summary: KafkaStorageException on reassignment when offline log directories exist Key: KAFKA-10314 URL: https://issues.apache.org/jira/browse/KAFKA-10314 Project: Kafka

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

2020-07-27 Thread Apache Jenkins Server
See

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Recommend Java 11 (#9080) [github] MINOR: INFO log4j when request re-join (#9068) -- [...truncated 2.77 MB...]

Re: [DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

2020-07-27 Thread Unmesh Joshi
Just checked etcd and zookeeper code, and both support leader to step down as a follower to make sure there are no two leaders if the leader has been disconnected from the majority of the followers For etcd this is https://github.com/etcd-io/etcd/issues/3866 For Zookeeper its

[jira] [Created] (KAFKA-10313) Out of range offset errors leading to offset reset

2020-07-27 Thread Varsha Abhinandan (Jira)
Varsha Abhinandan created KAFKA-10313: - Summary: Out of range offset errors leading to offset reset Key: KAFKA-10313 URL: https://issues.apache.org/jira/browse/KAFKA-10313 Project: Kafka

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

2020-07-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: INFO log4j when request re-join (#9068) -- [...truncated 2.80 MB...]

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

2020-07-27 Thread Bruno Cadonna
Hi Guozhang, Do you mean compression or compaction? Regarding compression, I agree with you except for the merge-sorting part. The docs describing what is stored in the block cache can be found under https://github.com/facebook/rocksdb/wiki/Block-Cache. Regarding compaction, my statement in

[DISCUSS] KIP-647: Add ability to handle late messages in streams-aggregation

2020-07-27 Thread Igor Piddubnyi
Hi everybody, I would like to start off the discussion for KIP-647: https://cwiki.apache.org/confluence/display/KAFKA/KIP-647%3A+Add+ability+to+handle+late+messages+in+streams-aggregation

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

2020-07-27 Thread Navinder Brar
+1 (non-binding). Thanks for the KIP, Bruno. ~Navinder On Friday, 24 July, 2020, 08:41:03 pm IST, John Roesler wrote: Thanks, Bruno! I'm +1 (binding) -John On Fri, Jul 24, 2020, at 07:04, Bruno Cadonna wrote: > Hi, > > After re-opening the discussion about > >

[VOTE] KIP-635: GetOffsetShell: support for multiple topics and consumer configuration override

2020-07-27 Thread Dániel Urbán
Hello everyone, I'd like to start a vote on KIP-635. The KIP enhances the GetOffsetShell tool by enabling querying multiple topic-partitions, adding new filtering options, and adding a config override option.

for help about kafka

2020-07-27 Thread 琴台之声
hello,apache kafka: I've already subscribed to Kafka® development. I am a user of kafka,the version I am using is kafka_2.12-2.1.1 now ,I want to update my kafka broker to a new version kafka_2.12-2.4.1,cause I know the new version has a lot of optimizations, such as improvement about kafka

[jira] [Resolved] (KAFKA-10301) Partition#remoteReplicasMap can be empty in certain race conditions

2020-07-27 Thread Stanislav Kozlovski (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10301?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stanislav Kozlovski resolved KAFKA-10301. - Resolution: Fixed > Partition#remoteReplicasMap can be empty in certain race