[jira] [Reopened] (KAFKA-15905) Restarts of MirrorCheckpointTask should not permanently interrupt offset translation

2024-05-23 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar reopened KAFKA-15905: --- reopening for backporting to 3.7.1 to be confermed > Restarts of MirrorCheckpointTask sho

[jira] [Created] (KAFKA-16622) Mirromaker2 first Checkpoint not emitted until consumer group fully catches up once

2024-04-25 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-16622: - Summary: Mirromaker2 first Checkpoint not emitted until consumer group fully catches up once Key: KAFKA-16622 URL: https://issues.apache.org/jira/browse/KAFKA-16622

Re: [VOTE] KIP-932: Queues for Kafka

2024-03-18 Thread Edoardo Comar
Thanks Andrew, +1 (binding) Edo On Mon, 18 Mar 2024 at 16:32, Kenneth Eversole wrote: > > Hi Andrew > > + 1 (Non-Binding) > > This will be great addition to Kafka > > On Mon, Mar 18, 2024 at 8:27 AM Apoorv Mittal > wrote: > > > Hi Andrew, > > Thanks for writing the KIP. This is indeed going

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-18 Thread Edoardo Comar
Thanks Manikumar, done and marked the issue as resolved On Mon, 18 Mar 2024 at 16:30, Manikumar wrote: > > Hi Edoardo, > > sure, pls go ahead and cherry-pick the changes to 3.7 and 3.6 branches. > > Thanks, > > On Mon, Mar 18, 2024 at 3:53 PM Edoardo Comar wrote: >

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-18 Thread Edoardo Comar
Hi Manikumar, https://issues.apache.org/jira/browse/KAFKA-16369 is merged in trunk now. can you please cherry-pick it to 3.6.2 ? I didn't see it included in the plan, would you like me to add it ? I'd consider it a blocker since Kafka may stay up and running but useless ... On Mon, 18 Mar 2024

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-14 Thread Edoardo Comar
Hi Manikumar, can we please include https://issues.apache.org/jira/browse/KAFKA-16369 in 3.6.2 ? While it's an issue still in trunk we actually discovered it our 3.6.1 systems thanks, Edo On Thu, 14 Mar 2024 at 08:39, Divij Vaidya wrote: > > Hi Manikumar, > > 1. Can you please take a look at

[jira] [Created] (KAFKA-16369) Broker may not shut down when SocketServer fails to bind as Address already in use

2024-03-13 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-16369: - Summary: Broker may not shut down when SocketServer fails to bind as Address already in use Key: KAFKA-16369 URL: https://issues.apache.org/jira/browse/KAFKA-16369

Re: [VOTE] KIP-971: Expose replication-offset-lag MirrorMaker2 metric

2024-01-18 Thread Edoardo Comar
Hi Elxan, +1 (binding). Thanks, Edo On Wed, 10 Jan 2024 at 14:01, Viktor Somogyi-Vass wrote: > > Hi Elxan, > > +1 (binding). > > Thanks, > Viktor > > On Mon, Jan 8, 2024 at 5:57 PM Dániel Urbán wrote: > > > Hi Elxan, > > +1 (non-binding) > > Thanks for the KIP, this will be a very useful

Re: [VOTE] KIP-993: Allow restricting files accessed by File and Directory ConfigProviders

2023-12-19 Thread Edoardo Comar
+1 (binding) Thanks for the KIP Tina! Edo On Tue, 19 Dec 2023 at 10:01, Gantigmaa Selenge wrote: > > Thank you Chris and Mickael for the vote! > > KIP is missing just one more +1 (binding). > > Regards, > Tina > > On Thu, Dec 14, 2023 at 1:48 PM Mickael Maison > wrote: > > > +1 (binding) > >

Re: Java 1.8 and TLSv1.3

2023-11-06 Thread Edoardo Comar
$.run(ConsoleConsumer.scala:77) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:54) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > > If I change the test in SslConfigs.java to test for presence of the TPSv1.3 > protocol rather than testing for Java

Re: Java 1.8 and TLSv1.3

2023-11-03 Thread Edoardo Comar
Andreas, do you mean that even if you configure your Java client running on Java8 with ssl.enabled.protocols=TLSv1.3 you can't connect to a Kafka broker using TLS1.3 ? On Sat, 28 Oct 2023 at 01:03, Ismael Juma wrote: > > Hi Andreas, > > The TLS code has run into changes in behavior across

Re: [ANNOUNCE] New committer: Yash Mayya

2023-09-21 Thread Edoardo Comar
Congratulations Yash On Thu, 21 Sept 2023 at 16:28, Bruno Cadonna wrote: > > Hi all, > > The PMC of Apache Kafka is pleased to announce a new Kafka committer > Yash Mayya. > > Yash's major contributions are around Connect. > > Yash authored the following KIPs: > > KIP-793: Allow sink connectors

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-09-05 Thread Edoardo Comar
d, 5 Jul 2023 at 16:55, Edoardo Comar wrote: > > Hi Jorge! > > On Fri, 30 Jun 2023 at 15:47, Jorge Esteban Quilcate Otoya > wrote: > > > > Thank you both for the replies! A couple more comments: > > > > The current proposal is to have ‘record.validation.policy’

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-07-05 Thread Edoardo Comar
Hi Jorge! On Fri, 30 Jun 2023 at 15:47, Jorge Esteban Quilcate Otoya wrote: > > Thank you both for the replies! A couple more comments: > > The current proposal is to have ‘record.validation.policy’ per topic > (default null). A flag would be something like > ‘record.validation.policy.enable’

[jira] [Resolved] (KAFKA-15144) MM2 Checkpoint downstreamOffset stuck to 1

2023-07-04 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar resolved KAFKA-15144. --- Resolution: Not A Bug Closing as not a bug. The "problem" arose as without conf

[jira] [Created] (KAFKA-15144) Checkpoint downstreamOffset stuck to 1

2023-07-03 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-15144: - Summary: Checkpoint downstreamOffset stuck to 1 Key: KAFKA-15144 URL: https://issues.apache.org/jira/browse/KAFKA-15144 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-15133) RequestMetrics MessageConversionsTimeMs count is ticked even when no conversion occurs

2023-06-28 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-15133: - Summary: RequestMetrics MessageConversionsTimeMs count is ticked even when no conversion occurs Key: KAFKA-15133 URL: https://issues.apache.org/jira/browse/KAFKA-15133

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-28 Thread Edoardo Comar
Hi Andrew, thanks for your comments ! Please see replies inline below. On Mon, 26 Jun 2023 at 16:51, Andrew Schofield wrote: > 4) For a new interface, I wonder whether it would be better to use > TopicIdPartition rather > than TopicPartition. Topic IDs are gradually spreading across the public

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-28 Thread Edoardo Comar
Hi Tom, thanks for tour comments, replies inline below. On Thu, 22 Jun 2023 at 10:58, Tom Bentley wrote: > > Hi Edorado and Adrian, > > Thanks for the KIP. > > I think it would be good to elaborate on exactly how validate() gets > called, because I think there are a number of potential

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-27 Thread Edoardo Comar
Hi Jorge thanks for the feedback. Comments inline below > 1. Similar to Kirk's first point, I'm also concerned on how would the > plugin developers / operators be able to apply multiple policies and how > configurations would be passed to each policy. We’ve only attempted to tackle the “one

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-26 Thread Edoardo Comar
dled and/or > outside the scope of this KIP? > > 3. What is the benefit to introducing the inner TopicMetadata and RecordProxy > interfaces vs. passing the TopicPartition, String (validation policy), and > Record into the validate() method directly? > > Thanks, > Kirk > >

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-20 Thread Edoardo Comar
Thanks Николай, We’d like to open a vote next week. Hopefully getting some more feedback before then. Edo On Wed, 7 Jun 2023 at 19:15, Николай Ижиков wrote: > Hello. > > As author of one of related KIPs I’m +1 for this change. > Long waited feature. > > > 7 июня 2023

Re: [ANNOUNCE] New committer: Divij Vaidya

2023-06-14 Thread Edoardo Comar
Congratulations Divij ! On Wed, 14 Jun 2023 at 10:51, Josep Prat wrote: > Congrats Divij! > > On Wed, Jun 14, 2023 at 11:45 AM Lucas Brutschy > wrote: > > > Congrats!! > > > > On Wed, Jun 14, 2023 at 11:02 AM Federico Valeri > > wrote: > > > > > > Congrats Divij! > > > > > > On Wed, Jun 14,

[DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-07 Thread Edoardo Comar
Dear all, Adrian and I would like to start a discussion thread on KIP-940: Broker extension point for validating record contents at produce time https://cwiki.apache.org/confluence/display/KAFKA/KIP-940%3A+Broker+extension+point+for+validating+record+contents+at+produce+time This KIP proposes a

[jira] [Created] (KAFKA-14996) CreateTopic falis with UnknownServerException if num partitions >= QuorumController.MAX_RECORDS_PER_BATCH

2023-05-12 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-14996: - Summary: CreateTopic falis with UnknownServerException if num partitions >= QuorumController.MAX_RECORDS_PER_BATCH Key: KAFKA-14996 URL: https://issues.apache.org/jira/bro

Re: [ANNOUNCE] New PMC chair: Mickael Maison

2023-05-02 Thread Edoardo Comar
Congratulations Mickael !! On Mon, 24 Apr 2023 at 20:18, Randall Hauch wrote: > Thank you, Jun, for all your contributions as PMC chair. > > And congratulations and thanks, Mickael, for volunteering to take over this > important role. > > Best regards, > Randall > > On Mon, Apr 24, 2023 at 1:39 

Re: KIP-922: Add the traffic metric of the partition dimension

2023-04-27 Thread Edoardo Comar
Hi hudeqi, thanks for the KIP. For the purpose of monitoring if partitions of a topic are used "fairly", the log end offset metric offers a good hint. Of course, it only expresses a message count, not bytes, but I find it sufficient and do not need the actual throughput per partition. Perhaps

Re: [DISCUSS] KIP-905: Broker interceptors

2023-02-21 Thread Edoardo Comar
this. This is a significant difference from one of the other KIPs mentioned, where the intention was for a producer to receive PolicyFailed exceptions. This should be part of the KIP, IMHO On Tue, 21 Feb 2023 at 13:53, Edoardo Comar wrote: > Hi David > thanks for the KIP. > >

Re: [DISCUSS] KIP-905: Broker interceptors

2023-02-21 Thread Edoardo Comar
Hi David thanks for the KIP. Two initial observations from me. I think the Rejected Alternatives section could compare your proposal to the prior art that you rightly mention initially. Also, the Java interface could extend Kafka's own Configurable. This allows an implementation to get hold of

[jira] [Created] (KAFKA-14657) Admin.fenceProducers fails when Producer has ongoing transaction - but Producer gets fenced

2023-01-27 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-14657: - Summary: Admin.fenceProducers fails when Producer has ongoing transaction - but Producer gets fenced Key: KAFKA-14657 URL: https://issues.apache.org/jira/browse/KAFKA-14657

[jira] [Resolved] (KAFKA-7666) KIP-391: Allow Producing with Offsets for Cluster Replication

2023-01-27 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7666?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar resolved KAFKA-7666. -- Resolution: Won't Fix KIP has been retired > KIP-391: Allow Producing with Offs

Re: [ANNOUNCE] New committer: Edoardo Comar

2023-01-10 Thread Edoardo Comar
gt; On 1/6/23 5:15 PM, Luke Chen wrote: > > > > > > >> Congratulations, Edoardo! > > > > > > >> > > > > > > >> Luke > > > > > > >> > > > > > > >> On Sat, Jan 7, 2023 at 7:58 AM Mick

[jira] [Created] (KAFKA-14571) ZkMetadataCache.getClusterMetadata is missing rack information for aliveNodes

2023-01-04 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-14571: - Summary: ZkMetadataCache.getClusterMetadata is missing rack information for aliveNodes Key: KAFKA-14571 URL: https://issues.apache.org/jira/browse/KAFKA-14571

RE: [VOTE] KIP-653: Upgrade log4j to log4j2

2022-03-23 Thread Edoardo Comar
Mickael, +1 from me - even if you didn't ask for a vote :-) Edo -- Edoardo Comar Event Streams for IBM Cloud From: Mickael Maison Sent: 23 March 2022 11:43 To: dev Subject: [EXTERNAL] Re: [VOTE] KIP-653

RE: [VOTE] KIP-653: Upgrade log4j to log4j2

2022-03-21 Thread Edoardo Comar
-- Edoardo Comar Event Streams for IBM Cloud From: Luke Chen Sent: 18 March 2022 07:57 To: dev Subject: [EXTERNAL] Re: [VOTE] KIP-653: Upgrade log4j to log4j2 Hi Dongjin, I know there are some discussions about

Re: [VOTE] KIP-719: Deprecate Log4J Appender

2022-01-20 Thread Edoardo Comar
Hi Dongjin, +1 (non-binding) many thanks Edoardo On Wed, 19 Jan 2022 at 14:07, Viktor Somogyi-Vass wrote: > Hi Dongjin, > > +1 (non-binding) from me as well. Thanks for picking this up! > > Viktor > > On Mon, Jan 17, 2022 at 1:07 PM Mickael Maison > wrote: > > > Hi Dongjin, > > > > +1

Re: [DISCUSS] KIP-660: Pluggable ReplicaAssignor

2020-12-03 Thread Edoardo Comar
Hi Colin thanks for your comments. I think your objections to creating an interface for replica placement could be used against similar server-side plug-ins (Authorizer, QuotaCallback). They too are on sensitive code paths, can cause problems if badly written/poorly tested and may cause a pain on

[jira] [Created] (KAFKA-10220) NPE when describing resources

2020-06-30 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-10220: - Summary: NPE when describing resources Key: KAFKA-10220 URL: https://issues.apache.org/jira/browse/KAFKA-10220 Project: Kafka Issue Type: Bug

RE: [VOTE] KIP 585: Filter and conditional SMTs

2020-05-19 Thread Edoardo Comar
+1 (non-binding) Thanks Tom -- Edoardo Comar Event Streams for IBM Cloud IBM UK Ltd, Hursley Park, SO21 2JN From: Gunnar Morling To: dev@kafka.apache.org Date: 19/05/2020 10:35 Subject:[EXTERNAL] Re: [VOTE] KIP 585: Filter

RE: [DISCUSS] Kafka 3.0

2020-05-06 Thread Edoardo Comar
Ryanne Dolan wrote on 05/05/2020 20:36:49: > Exactly. Why would 3.1 be the breaking release? No one would expect > everything to break going from 3.0 to 3.1 Agree completely > > Ryanne > > On Tue, May 5, 2020 at 2:34 PM Gwen Shapira wrote: > > > It sounds like the decision to make the next

Re: [VOTE] KIP-597: MirrorMaker2 internal topics Formatters

2020-05-04 Thread Edoardo Comar
+1 (non-binding) Thanks Mickael -- Edoardo Comar Event Streams for IBM Cloud IBM UK Ltd, Hursley Park, SO21 2JN From: Mickael Maison To: dev Date: 04/05/2020 11:45 Subject:[EXTERNAL] [VOTE] KIP-597: MirrorMaker2 internal

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

2020-02-06 Thread Edoardo Comar
+1 (non-binding) thanks for the KIP ! On Tue, 14 Jan 2020 at 13:57, Navinder Brar wrote: > +1 (non-binding) > Navinder > On Tuesday, 14 January, 2020, 07:24:02 pm IST, Ryanne Dolan < > ryannedo...@gmail.com> wrote: > > Bump. We've got 4 non-binding and one binding vote. > > Ryanne > > On

[jira] [Resolved] (KAFKA-8564) NullPointerException when loading logs at startup

2019-06-20 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8564?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar resolved KAFKA-8564. -- Resolution: Fixed Fix Version/s: 2.2.2 2.1.2 2.3.0

Re: [VOTE] KIP-419: Safely notify Kafka Connect SourceTask is stopped

2019-04-15 Thread Edoardo Comar
Thanks Andrew. +1 (non-binding) -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: Mickael Maison To: dev Date: 10/04/2019 10:14 Subject:Re: [VOTE] KIP-419: Safely notify Kafka Connect SourceTask

Re: [ANNOUNCE] New Committer: Bill Bejeck

2019-02-14 Thread Edoardo Comar
Well done Bill! -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: Rajini Sivaram To: dev Date: 14/02/2019 09:25 Subject:Re: [ANNOUNCE] New Committer: Bill Bejeck Congratulations, Bill! On Thu, Feb

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2019-02-04 Thread Edoardo Comar
replicating the value as-is - it would edit it by stripping the epoch. As previously mentioned, the __consumer_offsets topic does not need to be replicated by producing-with-offsets to it. -- Edoardo Comar IBM Event Streams On Wed, 23 Jan 2019

[jira] [Created] (KAFKA-7861) AlterConfig may change the source of another config entry if it matches the broker default

2019-01-23 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-7861: Summary: AlterConfig may change the source of another config entry if it matches the broker default Key: KAFKA-7861 URL: https://issues.apache.org/jira/browse/KAFKA-7861

Re: [ANNOUNCE] New Committer: Vahid Hashemian

2019-01-16 Thread Edoardo Comar
Bravo Vahid!!! Edo From: Mickael Maison To: Users Cc: dev Date: 16/01/2019 08:48 Subject:Re: [ANNOUNCE] New Committer: Vahid Hashemian Congratulations Vahid! Well done! On Wed, Jan 16, 2019 at 6:42 AM Dongjin Lee wrote: > > Congratulations, Vahid!! > > On Wed, Jan

Re: [VOTE] KIP-396: Add Commit/List Offsets Operations to AdminClient

2019-01-08 Thread Edoardo Comar
+1 (non-binding) Thanks Mickael! On Tue, 8 Jan 2019 at 17:39, Patrik Kleindl wrote: > +1 (non-binding) > Thanks, sounds very helpful > Best regards > Patrik > > > Am 08.01.2019 um 18:10 schrieb Mickael Maison >: > > > > Hi all, > > > > I'd like to start the vote on KIP-396: > > >

[VOTE] KIP-391: Allow Producing with Offsets for Cluster Replication

2019-01-08 Thread Edoardo Comar
-- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN Unless stated otherwise above: IBM United Kingdom Limited - Registered in England and Wales with number 741598. Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2019-01-07 Thread Edoardo Comar
Hi, I delayed starting the voting thread due to the festive period. I would like to start it this week. Has anyone any more feedback ? -- Edoardo Comar IBM Event Streams Edoardo Comar wrote on 13/12/2018 17:50:30: > From: Edoardo Co

Re: [VOTE] KIP-382 MirrorMaker 2.0

2018-12-24 Thread Edoardo Comar
+1 non-binding thanks for the KIP -- Edoardo Comar IBM Event Streams Harsha wrote on 21/12/2018 20:17:03: > From: Harsha > To: dev@kafka.apache.org > Date: 21/12/2018 20:17 > Subject: Re: [VOTE] KIP-382 MirrorMaker 2.0 >

[jira] [Resolved] (KAFKA-7761) CLONE - Add broker configuration to set minimum value for segment.bytes and segment.ms

2018-12-24 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar resolved KAFKA-7761. -- Resolution: Duplicate if there is a valid reason for a duplicate, feel free to repoen > CL

Re: [DISCUSS] KIP-402: Improve fairness in SocketServer processors

2018-12-14 Thread Edoardo Comar
number of client connections on a different listener. do you think it would make sense to also allow the `num.network.threads` to have an optional per-listener prefix ? ciao, Edo -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-12-13 Thread Edoardo Comar
Hi, as we haven't got any more feedback, we'd like to start a vote on KIP-391 on Monday https://cwiki.apache.org/confluence/display/KAFKA/KIP-391%3A+Allow+Producing+with+Offsets+for+Cluster+Replication -- Edoardo Comar IBM Event Streams IBM UK

[jira] [Created] (KAFKA-7720) kafka-configs script should also describe default broker entries

2018-12-11 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-7720: Summary: kafka-configs script should also describe default broker entries Key: KAFKA-7720 URL: https://issues.apache.org/jira/browse/KAFKA-7720 Project: Kafka

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-12-10 Thread Edoardo Comar
(shameless bump) any additional feedback is welcome ... thanks! Edoardo Comar wrote on 27/11/2018 15:35:09: > From: Edoardo Comar > To: dev@kafka.apache.org > Date: 27/11/2018 15:35 > Subject: Re: [DISCUSS] KIP-391: Allow Producing with Offsets for > Cluster Replication >

Re: [DISCUSS] KIP-396: Add Commit/List Offsets Operations to AdminClient

2018-11-30 Thread Edoardo Comar
-partition field in the Kafka ListOffsets Request API so you're not exposing the full APi power with the option -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: Mickael Maison To: dev Date: 30/11/2018 10:51

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-11-27 Thread Edoardo Comar
me offset be more generic? > > your producers could checkpoint the last (say log append) timestamp of > > records theyve seen, and when restoring in the remote site seek to > > those timestamps (which will be metadata in their committed offsets) - > > assumming replication takes

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-11-27 Thread Edoardo Comar
e this KIP seems to be designed fro active-passive > > > failover (there can be no produce traffic except the replicator) > > > wouldnt a solution based on seeking to a time offset be more generic? > > > your producers could checkpoint the last (say log append) timestamp of &

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-11-27 Thread Edoardo Comar
a in their committed offsets) - > assumming replication takes > 0 time you'd need to handle some dups, > but every kafka consumer setup needs to know how to handle those > anyway. can you please clarify? We do not expect any cooperation from users applications. thanks! E > On Fri, N

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-11-23 Thread Edoardo Comar
our interest ! cheers Edo ---------- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN Stanislav Kozlovski wrote on 22/11/2018 22:32:42: > From: Stanislav Kozlovski > To: dev@kafka.apache.org > Date: 22/11/2018 22:33 > Subject: R

Re: [DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-11-22 Thread Edoardo Comar
flag is needed to distinguish a batch with a desired base offset of 0, from a regular batch for which offsets need to be generated. I would not restrict a principal to only send-with-offsets (by making that mandatory via the ACL). Thanks Edo & Mickael ------

Re: [ANNOUNCE] Apache Kafka 2.1.0

2018-11-21 Thread Edoardo Comar
e, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank, > > Target, The New York Times, Uber, Yelp, and Zalando, among others. > > > > A big thank you for the following 100 contributors to this release! > > > > Ahmed Al Mehdi, Aleksei Izmalkin, Alex Dunayevsky, Amit Sel

[jira] [Created] (KAFKA-7666) KIP-391: Allow Producing with Offsets for Cluster Replication

2018-11-21 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-7666: Summary: KIP-391: Allow Producing with Offsets for Cluster Replication Key: KAFKA-7666 URL: https://issues.apache.org/jira/browse/KAFKA-7666 Project: Kafka

[DISCUSS] KIP-391: Allow Producing with Offsets for Cluster Replication

2018-11-21 Thread Edoardo Comar
-- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN Unless stated otherwise above: IBM United Kingdom Limited - Registered in England and Wales with number 741598. Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire

Re: [VOTE] KIP-380: Detect outdated control requests and bounced brokers using broker generation

2018-10-31 Thread Edoardo Comar
Thanks for the KIP. +1 (non-binding) -- Edoardo Comar IBM Event Streams From: Harsha Chintalapani To: Patrick Huang , dev@kafka.apache.org Cc: "dev@kafka.apache.org" Date: 30/10/2018 19:11 Subject:Re: [VOT

Re: [ANNOUNCE] New Committer: Manikumar Reddy

2018-10-12 Thread Edoardo Comar
Well done Manikumar ! -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: "Matthias J. Sax" To: dev Cc: users Date: 11/10/2018 23:41 Subject:Re: [ANNOUNCE] New Committer: Manik

Re: New release branch 2.1.0

2018-10-11 Thread Edoardo Comar
Thanks Rajini for the PR review, as agreed we have updated KIP-302 to match the code so the new config value for the entry client.dns.lookup is "use_all_dns_ips" (underscores instead of dot separators) ------ Edoardo Comar IBM Event Stre

Re: [VOTE] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-10-11 Thread Edoardo Comar
Thanks Rajini for the PR review, as agreed we have updated KIP-302 to match the code so the new config value for the entry client.dns.lookup is "use_all_dns_ips" (underscores instead of dot separators) cheers Edo ------ Edoardo Comar

Re: [DISCUSS] Apache Kafka 2.1.0 Release Plan

2018-09-25 Thread Edoardo Comar
Hi Dong many thanks for driving the release! KIP-81 previously voted as adopted has a ready-to-review JIRA and PR. Shall we just amend the wiki ? -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: Dong Lin

Re: [ANNOUNCE] New committer: Colin McCabe

2018-09-25 Thread Edoardo Comar
Congratulations Colin ! -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: Ismael Juma To: Kafka Users , dev Date: 25/09/2018 09:40 Subject:[ANNOUNCE] New committer: Colin McCabe Hi all, The PMC

Re: [VOTE] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-25 Thread Edoardo Comar
Many thanks to all that voted, This KIP has passed the vote - within the COB of 24/09/2018 - with 3 binding votes (Rajini, Damian, Gwen) and 4 non-binding votes (Eno, Jonathan and implicitly :-) Mickael and me) Edo -- Edoardo Comar IBM Event

RE: [VOTE] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-23 Thread Edoardo Comar
bumping the thread as the KIP needs 2 more binding votes ... pretty please ... -- Edoardo Comar IBM Event Streams IBM UK Ltd, Hursley Park, SO21 2JN From: "Skrzypek, Jonathan" To: "dev@kafka.apache.org" Date: 20/0

RE: [VOTE] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-20 Thread Edoardo Comar
Hi Jonathan we'll update the PR for KIP-302 soon. We do not need KIP-235 actually, they only share the name of the configuration entry. thanks Edo PS - we need votes :-) -- Edoardo Comar IBM Message Hub IBM UK Ltd, Hursley Park, SO21 2JN

Re: [VOTE] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-18 Thread Edoardo Comar
+1 (non binding) Thanks Ron & Rajini ! -- Edoardo Comar IBM Message Hub IBM UK Ltd, Hursley Park, SO21 2JN From: Ron Dagostino To: dev@kafka.apache.org Date: 18/09/2018 16:09 Subject:[VOTE] KIP 368: Allow SASL Connect

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-18 Thread Edoardo Comar
Yes I should, should't I ? :-) -- Edoardo Comar IBM Message Hub IBM UK Ltd, Hursley Park, SO21 2JN From: Rajini Sivaram To: dev Date: 18/09/2018 15:33 Subject:Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS

[VOTE] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-18 Thread Edoardo Comar
-- Edoardo Comar IBM Message Hub IBM UK Ltd, Hursley Park, SO21 2JN Unless stated otherwise above: IBM United Kingdom Limited - Registered in England and Wales with number 741598. Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-18 Thread Edoardo Comar
the brokers are up, the cluster is fully functioning even if just one of the LBs is up. HTH, Edo -- Edoardo Comar IBM Message Hub IBM UK Ltd, Hursley Park, SO21 2JN From: Eno Thereska To: dev@kafka.apache.org Date: 18/09/2018 10:24 Subject

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-09-11 Thread Edoardo Comar
May 2018 at 12:36, Edoardo Comar wrote: > Hi Jonathan, > I'm ok with an expandable enum for the config that could be extended > in the future. > It is marginally better than multiple potentially conflicting config > entries. > > Though as I think the change for KIP-302 is ind

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-06-28 Thread Edoardo Comar
be able to know whether a topic exists. Edoardo -- Edoardo Comar IBM Message Hub IBM UK Ltd, Hursley Park, SO21 2JN From: Guozhang Wang To: dev@kafka.apache.org Date: 27/06/2018 19:42 Subject:Re: [VOTE] KIP-277 - Fine Grained ACL

[jira] [Created] (KAFKA-6994) KafkaConsumer.poll throwing AuthorizationException timeout-dependent

2018-06-05 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-6994: Summary: KafkaConsumer.poll throwing AuthorizationException timeout-dependent Key: KAFKA-6994 URL: https://issues.apache.org/jira/browse/KAFKA-6994 Project: Kafka

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-25 Thread Edoardo Comar
LED? > > Ismael > > On Wed, Apr 25, 2018 at 2:45 AM Edoardo Comar <eco...@uk.ibm.com> wrote: > >> Hi, >> >> The discuss thread on KIP-277 ( >> https://www.mail-archive.com/dev@kafka.apache.org/msg86540.html ) >> seems to have been fruitful

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-25 Thread Edoardo Comar
- dns.lookup.behaviour : bootstrap.servers, advertised.listeners, both > > Thoughts ? > > Jonathan Skrzypek > > > -Original Message- > From: Edoardo Comar [mailto:edoco...@gmail.com] > Sent: 17 May 2018 23:50 > To: dev@kafka.apache.org > Subject: Re: [DISCUSS]

Re: [VOTE] KIP-306: Configuration for Delaying Response to Failed Client Authentication

2018-05-21 Thread Edoardo Comar
+1 (non-binding) Thanks ! On 21 May 2018 at 19:18, Bill Bejeck wrote: > Thanks for the KIP. > > +1 > > -Bill > > On Mon, May 21, 2018 at 1:41 PM, Ted Yu wrote: > >> +1 >> >> On Mon, May 21, 2018 at 9:02 AM, Guozhang Wang wrote: >> >>

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-21 Thread Edoardo Comar
ian@gmail.com> wrote: > +1 (binding) > > Thanks > > On Mon, 21 May 2018 at 11:26 Edoardo Comar <edoco...@gmail.com> wrote: > >> Hi, KIP freeze is tomorrow. Anyone else would like to cast their vote ? >> >> voting so far : >> binding: Guozha

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-21 Thread Edoardo Comar
Hi, KIP freeze is tomorrow. Anyone else would like to cast their vote ? voting so far : binding: Guozhang, Rajini non-binding: Ted, Stephane, Vahid, Colin, Manikumar, Matt On 18 May 2018 at 09:38, Edoardo Comar <edoco...@gmail.com> wrote: > bumping again as this KIP is still missing on

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-18 Thread Edoardo Comar
:26 AM, Edoardo Comar <edoco...@gmail.com> wrote: > >> Hi, >> bumping the thread as the current vote count for this KIP is >> 2 binding +1 >> 5 non-binding +1 >> >> thanks, Edo >> >> On 8 May 2018 at 16:14, Edoardo Comar <edoco...@gmail.com>

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-17 Thread Edoardo Comar
re consistency so that a user can decide to use cnames or > not (depending on their certificates and Kerberos principals in their > environment) and it would work. > > Jonathan Skrzypek > > -Original Message- > From: Edoardo Comar [mailto:edoco...@gmail.com] >

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-16 Thread Edoardo Comar
gt; Jonathan Skrzypek > > -Original Message- > From: Edoardo Comar [mailto:edoco...@gmail.com] > Sent: 16 May 2018 17:03 > To: dev@kafka.apache.org > Subject: Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved > IP addresses > > Hi Rajini, >

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-16 Thread Edoardo Comar
h > the load balancers. In Jonathan's case, I presume he has Kerberos > principals for the equivalent of tlp-eu-west.apache.org and > themis.apache.org. We would want to support both scenarios regardless of > the security protocol, just need to come up with configuration options tha

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-15 Thread Edoardo Comar
Hi, bumping the thread as the current vote count for this KIP is 2 binding +1 5 non-binding +1 thanks, Edo On 8 May 2018 at 16:14, Edoardo Comar <edoco...@gmail.com> wrote: > Hi, > bumping the thread as the current vote count for this KIP is > 2 binding +1 > 5 non-binding +1 &g

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Edoardo Comar
lso solve what this KIP is trying to achieve. >> >> When parsing bootstrap.servers, KIP-235 has the client add all underlying >> hostnames and IPs. >> And this happens before hitting the NetworkClient. >> >> So to me the client will try every single endpoint be

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Edoardo Comar
will try every single endpoint behind any > bootstrap.servers record. > > See > https://github.com/apache/kafka/pull/4485/commits/24757eb7b06bcf8c7d7649c85232c52b5d54f0e4#diff-89ef153462e64c250a21bd324ae1a851 > which calls getAllByName like you suggested > > Jonathan Skrzypek

[DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Edoardo Comar
Hi all, We just opened a KIP to add support for the client to use all IPs returned by DNS for the brokers The details are here - https://cwiki.apache.org/confluence/display/KAFKA/KIP-302+-+Enable+Kafka+clients+to+use+all+DNS+resolved+IP+addresses The JIRA and provisional PR (where the

Re: [VOTE] KIP-266: Add TimeoutException for KafkaConsumer#position

2018-05-10 Thread Edoardo Comar
+1 (non-binding) On 10 May 2018 at 10:29, zhenya Sun wrote: > +1 non-binding > > > 在 2018年5月10日,下午5:19,Manikumar 写道: > > > > +1 (non-binding). > > Thanks. > > > > On Thu, May 10, 2018 at 2:33 PM, Mickael Maison < > mickael.mai...@gmail.com> > >

Re: [VOTE] KIP-283: Efficient Memory Usage for Down-Conversion

2018-05-10 Thread Edoardo Comar
+1 (non-binding) On 10 May 2018 at 09:56, Rajini Sivaram wrote: > Hi Dhruvil, Thanks for the KIP! > > +1 (binding) > > Regards, > > Rajini > > On Wed, May 9, 2018 at 9:28 PM, Dhruvil Shah wrote: > > > Thanks for the feedback, Jason and Ismael. I

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-10 Thread Edoardo Comar
+1 (non-binding) On 10 May 2018 at 09:36, Manikumar wrote: > +1 (non-binding) > > Thanks. > > On Wed, May 9, 2018 at 10:09 PM, Mickael Maison > wrote: > > > +1, thanks for the KIP! > > > > On Wed, May 9, 2018 at 4:41 PM, Ted Yu

Re: [VOTE] KIP-277 - Fine Grained ACL for CreateTopics API

2018-05-08 Thread Edoardo Comar
018 at 9:59 PM, Colin McCabe <cmcc...@apache.org> wrote: > > > +1 (non-binding) > > > > best, > > Colin > > > > > > On Wed, Apr 25, 2018, at 02:45, Edoardo Comar wrote: > > > Hi, > > > > > > The discuss thread on KIP-277

[jira] [Created] (KAFKA-6863) Kafka clients should try to use multiple DNS resolved IP addresses if the first one fails

2018-05-04 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-6863: Summary: Kafka clients should try to use multiple DNS resolved IP addresses if the first one fails Key: KAFKA-6863 URL: https://issues.apache.org/jira/browse/KAFKA-6863

  1   2   3   4   >