[jira] [Resolved] (KAFKA-7925) Constant 100% cpu usage by all kafka brokers

2019-12-12 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7925?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7925. --- Fix Version/s: 2.5.0 Reviewer: Guozhang Wang Resolution: Fixed > Constant 100

Re: [DISCUSS] KIP-489 Kafka Consumer Record Latency Metric

2019-12-12 Thread Habib Nahas
Hi Sean, Thanks for the KIP. As I understand it users are free to set their own timestamp on ProducerRecord. What is the recommendation for the proposed metric in a scenario where the user sets this timestamp in timezone A and consumes the record in timezone B. Its not clear to me if a custom

Re: [VOTE] KIP-409: Allow creating under-replicated topics and partitions

2019-12-12 Thread Mickael Maison
Bumping this thread, I've not seen any votes or feedback. On Wed, Nov 13, 2019 at 12:17 PM Mickael Maison wrote: > > Hi all, > > I'd like to start a vote on KIP-409: Allow creating under-replicated > topics and partitions > https://cwiki.apache.org/confluence/display/KAFKA/KIP-409%3A+Allow+creati

Re: [VOTE] KIP-518: Allow listing consumer groups per state

2019-12-12 Thread Mickael Maison
Bumping this thread On Tue, Nov 19, 2019 at 6:27 AM Kevin Lu wrote: > > +1 (non-binding) > > I can see this being useful. Thanks for the KIP! > > Regards, > Kevin > > On Mon, Nov 18, 2019 at 4:09 AM Mickael Maison > wrote: > > > Hi all, > > > > I'd like to start the vote on KIP-518: > > > > http

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2019-12-12 Thread Mickael Maison
+1 (binding) Thanks for the KIP! On Thu, Dec 5, 2019 at 12:56 AM Ryanne Dolan wrote: > > Bump. We've got 2 non-binding votes so far. > > On Wed, Nov 13, 2019 at 3:32 PM Ning Zhang wrote: > > > My current plan is to implement this in "MirrorCheckpointTask" > > > > On 2019/11/02 03:30:11, Xu Jianh

Re: [VOTE] 2.4.0 RC4

2019-12-12 Thread Israel Ekpo
@Manikumar , I have rerun the latest release candidate (RC4) and all the tests have passed with no errors. Should we go ahead and close this issue regarding the Flaky test? - https://issues.apache.org/jira/browse/KAFKA-9283 On Tue, Dec 10, 2019 at 11:11 AM Adam Bellemare wrote: > - All PGP sig

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

2019-12-12 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7925 - Create credentials only once for sun.security.jgss.native -- [...truncated 5.58 MB...] org.apache.kafka.streams.TopologyTestDriverTest

Re: [VOTE] 2.4.0 RC4

2019-12-12 Thread Manikumar
Hi Israel, Thanks for testing the release candidate. We can use https://issues.apache.org/jira/browse/KAFKA-9283 for fixing the flaky test. Thanks, On Thu, Dec 12, 2019 at 8:18 PM Israel Ekpo wrote: > > @Manikumar , I have rerun the latest release > candidate (RC4) and all the tests have pass

Re: [VOTE] 2.4.0 RC4

2019-12-12 Thread Manikumar
Hi all, Successful Jenkins builds for the 2.4 branch: Unit/integration tests: https://builds.apache.org/blue/organizations/jenkins/kafka-2.4-jdk8/detail/kafka-2.4-jdk8/106/ System tests: http://confluent-kafka-2-4-system-test-results.s3-us-west-2.amazonaws.com/2019-12-12--001.1576143096--confluent

Re: [VOTE] 2.4.0 RC4

2019-12-12 Thread Mickael Maison
Thanks Manikumar for running this release I verified the signatures All tests passed without failures. We've also been running this release for a few days in a couple of environments without any issues. +1 (binding) On Thu, Dec 12, 2019 at 3:14 PM Manikumar wrote: > > Hi all, > > Successful Jen

Re: [VOTE] 2.4.0 RC4

2019-12-12 Thread Israel Ekpo
Thanks @Manikumar for running the release as well. In my latest run, all signature validations and tests passed successfully without any issues. +1 (non-binding) On Thu, Dec 12, 2019 at 10:25 AM Mickael Maison wrote: > Thanks Manikumar for running this release > > I verified the signatures >

[jira] [Created] (KAFKA-9294) Enhance DSL Naming Guide to Include All Naming Rules

2019-12-12 Thread Bill Bejeck (Jira)
Bill Bejeck created KAFKA-9294: -- Summary: Enhance DSL Naming Guide to Include All Naming Rules Key: KAFKA-9294 URL: https://issues.apache.org/jira/browse/KAFKA-9294 Project: Kafka Issue Type: Im

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2019-12-12 Thread Andrew Schofield
+1 (non-binding) On 12/12/2019, 14:20, "Mickael Maison" wrote: +1 (binding) Thanks for the KIP! On Thu, Dec 5, 2019 at 12:56 AM Ryanne Dolan wrote: > > Bump. We've got 2 non-binding votes so far. > > On Wed, Nov 13, 2019 at 3:32 PM Ning Zhang wrote: >

[jira] [Created] (KAFKA-9295) KTableKTableForeignKeyInnerJoinMultiIntegrationTest#shouldInnerJoinMultiPartitionQueryable

2019-12-12 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-9295: -- Summary: KTableKTableForeignKeyInnerJoinMultiIntegrationTest#shouldInnerJoinMultiPartitionQueryable Key: KAFKA-9295 URL: https://issues.apache.org/jira/browse/KAFKA-9295

Re: [VOTE] KIP-514 Add a bounded flush() API to Kafka Producer

2019-12-12 Thread radai
so can we call this passed ? On Thu, Nov 7, 2019 at 7:43 AM Satish Duggana wrote: > > +1 (non-binding) > > On Thu, Nov 7, 2019 at 8:58 PM Ismael Juma wrote: > > > > +1 (binding) > > > > On Thu, Oct 24, 2019 at 9:33 PM radai wrote: > > > > > Hello, > > > > > > I'd like to initiate a vote on KIP-

Re: [EXTERNAL] Re: [DISCUSS] KIP-280: Enhanced log compaction

2019-12-12 Thread radai
may I suggest that if, under "header" strategy, multiple records are found with identical header values they are ALL kept? this would be useful in cases where users send larger payloads than max record size to kafka and are forced to fragment them - by setting the same header in all fragments it wo

[DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

2019-12-12 Thread Konstantine Karantasis
I've taken a second look to KIP-158 after syncing with Randall Hauch, who was the original author of the proposal, and I have updated the KIP in place. The main new features of this updated KIP-158 is the introduction of groups of configs that can be composed and the ability to match topics to the

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

2019-12-12 Thread Randall Hauch
Thanks for updating and improving this KIP, Konstantine! Overall I'm pretty happy with the KIP, but I have a few fairly nit-picky questions/comments. 1) Will connectors see these configuration properties when Connect calls Connector.start(...)? We've added a number of configs to connectors that ar

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

2019-12-12 Thread Ryanne Dolan
Konstantine, thanks for the updates. I wonder if we should take your proposal one step further and make this pluggable. Your include/exclude regexes are great out-of-the-box features, but it may be valuable to plug-in more sophisticated logic to handle topic creation. Instead of enabling/disabling

Re: [VOTE] KIP-409: Allow creating under-replicated topics and partitions

2019-12-12 Thread Ryanne Dolan
+1 non-binding, thx On Thu, Dec 12, 2019 at 6:09 AM Mickael Maison wrote: > Bumping this thread, I've not seen any votes or feedback. > > On Wed, Nov 13, 2019 at 12:17 PM Mickael Maison > wrote: > > > > Hi all, > > > > I'd like to start a vote on KIP-409: Allow creating under-replicated > > top

Re: [VOTE] KIP-518: Allow listing consumer groups per state

2019-12-12 Thread Ryanne Dolan
+1 non-binding, thx On Thu, Dec 12, 2019 at 6:09 AM Mickael Maison wrote: > Bumping this thread > > On Tue, Nov 19, 2019 at 6:27 AM Kevin Lu wrote: > > > > +1 (non-binding) > > > > I can see this being useful. Thanks for the KIP! > > > > Regards, > > Kevin > > > > On Mon, Nov 18, 2019 at 4:09 A

Re: [DISCUSS] KIP-489 Kafka Consumer Record Latency Metric

2019-12-12 Thread Sean Glover
Hi Habib, Thanks for question! If the consumer is in a different timezone than the timezone used to produce messages to a partition then you can use an implementation of LatencyTime to return the current time of that timezone. The current design assumes that messages produced to a partition must a

Build failed in Jenkins: kafka-2.2-jdk8-old #195

2019-12-12 Thread Apache Jenkins Server
See Changes: [bill] KAFKA-8705: Remove parent node after leaving loop to prevent NPE (#7117) -- Started by an SCM change Started by an SCM change Running as SYSTEM [EnvI

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

2019-12-12 Thread Apache Jenkins Server
See

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

2019-12-12 Thread Apache Jenkins Server
See Changes: [bill] KAFKA-8705: Remove parent node after leaving loop to prevent NPE (#7117) -- [...truncated 2.96 MB...] kafka.controller.ControllerChannelManagerTest > te

Re: [VOTE] KIP-514 Add a bounded flush() API to Kafka Producer

2019-12-12 Thread Gwen Shapira
You got 3 binding votes (Joel, Harsha, Ismael) - the vote passed on Nov 7. Happy hacking! On Thu, Dec 12, 2019 at 11:35 AM radai wrote: > > so can we call this passed ? > > On Thu, Nov 7, 2019 at 7:43 AM Satish Duggana > wrote: > > > > +1 (non-binding) > > > > On Thu, Nov 7, 2019 at 8:58 PM Is

Re: [DISCUSS] KIP-489 Kafka Consumer Record Latency Metric

2019-12-12 Thread Sean Glover
Hi, After my last reply I had a nagging feeling something wasn't right, and I remembered that epoch time is UTC. This makes the discussion about timezone irrelevant, since we're always using UTC. This makes the need for the LatencyTime interface that I proposed in the design irrelevant as well,

[jira] [Created] (KAFKA-9296) Correlation id for response () does not match request ()

2019-12-12 Thread Enhon Bryant (Jira)
Enhon Bryant created KAFKA-9296: --- Summary: Correlation id for response () does not match request () Key: KAFKA-9296 URL: https://issues.apache.org/jira/browse/KAFKA-9296 Project: Kafka Issue Ty

Re: [VOTE] 2.4.0 RC4

2019-12-12 Thread Gwen Shapira
+1 (binding) Validated signatures, tests and ran some test workloads. Thank you so much for driving this. Mani. On Mon, Dec 9, 2019 at 9:32 AM Manikumar wrote: > > Hello Kafka users, developers and client-developers, > > This is the fifth candidate for release of Apache Kafka 2.4.0. > > This re

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

2019-12-12 Thread Apache Jenkins Server
See Changes: [matthias] KAFKA-6049: Add time window support for cogroup (#7774) [github] KAFKA-8705: Remove parent node after leaving loop to prevent NPE (#7117) -- [..

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2019-12-12 Thread Tom Bentley
+1 (non-binding) On Thu, Dec 12, 2019 at 6:33 PM Andrew Schofield wrote: > +1 (non-binding) > > On 12/12/2019, 14:20, "Mickael Maison" wrote: > > +1 (binding) > Thanks for the KIP! > > On Thu, Dec 5, 2019 at 12:56 AM Ryanne Dolan > wrote: > > > > Bump. We've got 2 non-bind