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

2020-08-31 Thread Dániel Urbán
ument name. Overall, I believe that this is a non-intrusive change - minor improvements without breaking changes. But for some, this would be a great improvement in using Kafka. Thank you in advance, Daniel Dániel Urbán ezt írta (időpont: 2020. aug. 27., Cs, 17:52): > Hi all, > > Please

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

2020-10-08 Thread Dániel Urbán
Thank you for the votes! The KIP passes with 3 binding votes (Manikumar, Mickael, Ismael) and 3 non-binding votes (Viktor, Kamal, David). Daniel Ismael Juma ezt írta (időpont: 2020. okt. 8., Cs, 16:27): > Thanks for the KIP, +1 (binding). > > On Mon, Jul 27, 2020 at 1:09 AM Dán

confluence access

2020-10-13 Thread Dániel Urbán
Hi, I'd like to access the wiki and contribute to KIPs, please add me. My username is urbandan. Thanks in advance, Daniel

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

2020-08-18 Thread Dániel Urbán
Hello everyone, Please, if you are interested in this KIP and PR, don't forget to vote. Thank you, Daniel Dániel Urbán ezt írta (időpont: 2020. aug. 13., Cs, 14:00): > Hi David, > > Thank you for the suggestion. KIP-635 was referencing the --broker-list > issue, but based on you

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

2020-08-27 Thread Dániel Urbán
Hi all, Please vote if you'd like to see this implemented. This one fixes a long-time debt, would be nice to see it pass. Thank you, Daniel Dániel Urbán ezt írta (időpont: 2020. aug. 18., K, 14:06): > Hello everyone, > > Please, if you are interested in this KIP and PR, don't forge

Re: [DISCUSS] KIP-567: Kafka Cluster Audit (new discussion)

2020-09-22 Thread Dániel Urbán
this to see > how it looks like and whether it solves the problem. To be honest I think > it would be better than publishing the request classes as an API because > here we're restricting access to only what is necessary. > > Thanks, > Viktor > > > > On Fri, Sep 18, 202

Re: [DISCUSS] KIP-567: Kafka Cluster Audit (new discussion)

2020-10-01 Thread Dániel Urbán
any specific ideas about the implementation? > > Viktor > > On Tue, Sep 22, 2020 at 9:05 AM Dániel Urbán > wrote: > > > An example I had in mind was the ProduceResponse - the auditor might need > > access to the new end offset of the partitions. > > The event

Re: [DISCUSS] KIP-567: Kafka Cluster Audit (new discussion)

2020-09-18 Thread Dániel Urbán
Hi, Thanks for the KIP. If the auditor needs access to the details of the action, one could argue that even the response should be passed down to the auditor. Is it feasible to convert the Java requests and responses to public API? If not, do we have another option to access this info in the

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

2020-07-30 Thread Dániel Urbán
Hi everyone, If you are interested in this KIP, please do not forget to vote. Thanks, Daniel Viktor Somogyi-Vass ezt írta (időpont: 2020. júl. 28., K, 16:06): > +1 from me (non-binding), thanks for the KIP. > > On Mon, Jul 27, 2020 at 10:02 AM Dániel Urbán > wrote: > >

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

2020-08-13 Thread Dániel Urbán
that PR asking if they plan to continue it. If > not, > we could do it as part of your PR directly. > > Cheers, > David > > On Mon, Aug 10, 2020 at 10:49 AM Dániel Urbán > wrote: > > > Hi everyone, > > > > Just a reminder, please vote if you are

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

2020-08-10 Thread Dániel Urbán
Hi everyone, Just a reminder, please vote if you are interested in this KIP being implemented. Thanks, Daniel Dániel Urbán ezt írta (időpont: 2020. júl. 31., P, 9:01): > Hi David, > > There is another PR linked on KAFKA-8507, which is still open: > https://github.com/apache/kaf

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
eed a separate option to pass an consumer property. > fewer options are better. > > Thanks, > Manikumar > > On Wed, Jun 24, 2020 at 8:53 PM Dániel Urbán > wrote: > > > Hi, > > > > I see that this KIP turned somewhat inactive - I'd like to pick it up and > > w

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
KIP with the same id: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=85474993 What is the protocol in this case? Should I acquire a new id for the GetOffsetShell KIP, and update it? Thanks in advance, Daniel Dániel Urbán ezt írta (időpont: 2020. jún. 30., K, 9:23): >

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
> Yes, we can assign new id to this KIP. > > Thanks. > > On Tue, Jun 30, 2020 at 6:59 PM Dániel Urbán > wrote: > > > Hi, > > > > To help with the discussion, I also have a PR for this KIP now. > reflecting > > the current state of the KIP: https

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-30 Thread Dániel Urbán
tool output if it has DESCRIBE > privileges for some topics but hasn't for others? > > ________ > 发件人: Dániel Urbán > 发送时间: 2020年6月30日 22:15 > 收件人: dev@kafka.apache.org > 主题: Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support > for

contributor request

2020-06-24 Thread Dániel Urbán
Hi, I'd like to work on some KIPs. Can you please add me to the contributors? My username is durban on both JIRA and Confluence. Thanks in advance, Daniel

Re: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-06-24 Thread Dániel Urbán
Hi, I see that this KIP turned somewhat inactive - I'd like to pick it up and work on it if it is okay. Part of the work is done, as switching to the Consumer API is already in trunk, but some functionality is still missing. I've seen the current PR and the discussion so far, only have a few

[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.

Re: 回复: [DISCUSS] KIP-308: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimize the number of requests to server

2020-07-21 Thread Dániel Urbán
have a question about the > multiple-topic lookup situation. > > In a secured environment, what does the tool output if it has DESCRIBE > privileges for some topics but hasn't for others? > > > 发件人: Dániel Urbán > 发送时间: 2020年6月30日 22

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

2020-07-31 Thread Dániel Urbán
inding), thanks for the KIP! > > > > On Thu, Jul 30, 2020 at 3:31 PM Manikumar > > wrote: > > > > > +1 (binding) > > > > > > Thanks for the KIP! > > > > > > > > > > > > On Thu, Jul 30, 2020 at 3:07 PM Dániel Urbán > &

[DISCUSS] KIP-709: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2021-01-26 Thread Dániel Urbán
Hello everyone, I would like to start a discussion on KIP-709, which addresses some missing features in MM2 dedicated mode. Currently, the dedicated mode of MM2 does not fully support running in a cluster. The core issue is that the Connect REST Server is not included in the dedicated mode,

Re: [DISCUSS] KIP-709: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2021-01-26 Thread Dániel Urbán
And I guess providing the link wouldn't hurt either: https://cwiki.apache.org/confluence/display/KAFKA/KIP-709%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters On Tue, Jan 26, 2021 at 9:35 AM Dániel Urbán wrote: > Hello everyone, > > I would like to start a d

Re: [DISCUSS] KIP-709: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2021-01-26 Thread Dániel Urbán
o KIP-710? > > Thanks > > Tom > > > On Tue, Jan 26, 2021 at 8:35 AM Dániel Urbán > wrote: > > > Hello everyone, > > > > I would like to start a discussion on KIP-709, which addresses some > missing > > features in MM2 dedicated mode. > > &g

[DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2021-01-26 Thread Dániel Urbán
Hello everyone, I would like to start a discussion on KIP-709, which addresses some missing features in MM2 dedicated mode. https://cwiki.apache.org/confluence/display/KAFKA/KIP-710%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters Currently, the dedicated mode of MM2

Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2021-02-18 Thread Dániel Urbán
Hello everyone, * Sorry, I meant KIP-710. Right now the MirrorMaker cluster is somewhat unreliable, and not supporting running in a cluster properly. I'd say that fixing this would be a nice addition. Does anyone have some input on this? Thanks in advance Daniel Dániel Urbán ezt írta (időpont

Re: Transactions, delivery timeout and changing transactional producer behavior

2022-07-07 Thread Dániel Urbán
ur PR! :) > > Thank you. > Luke > > On Thu, Jul 7, 2022 at 3:07 AM Artem Livshits > wrote: > > > Hi Daniel, > > > > What you say makes sense. Could you file a bug and put this info there > so > > that it's easier to track? > > > > -A

Re: Transactions, delivery timeout and changing transactional producer behavior

2022-07-08 Thread Dániel Urbán
that the producer has to be recreated after a delivery timeout. I think that is still fine compared to the out-of-order messages. Looking forward to your reviews, Daniel Dániel Urbán ezt írta (időpont: 2022. júl. 7., Cs, 12:04): > Thanks for the feedback, I created > https://issues.apache.org/jira/

Transactions, delivery timeout and changing transactional producer behavior

2022-07-06 Thread Dániel Urbán
Hello everyone, I've been investigating some transaction related issues in a very problematic cluster. Besides finding some interesting issues, I had some ideas about how transactional producer behavior could be improved. My suggestion in short is: when the transactional producer encounters an

Re: Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2022-08-25 Thread Dániel Urbán
ill affect the direction of the design discussion too much, but > it does help strengthen the motivation. > > Cheers, > > Chris > > On 2021/02/18 15:57:36 Dániel Urbán wrote: > > Hello everyone, > > > > * Sorry, I meant KIP-710. > > > > Right now the MirrorMa

Re: Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2022-08-29 Thread Dániel Urbán
ss any security concerns. My instinct would be to address this in a > follow-up KIP in order to reduce scope creep, though, and keep this KIP > focused on addressing the bug with multi-node dedicated MM2 clusters. What > do you think? > > Cheers, > > Chris > > [1] - >

Re: Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2022-09-27 Thread Dániel Urbán
.apache.org/confluence/display/KAFKA/KIP-507%3A+Securing+Internal+Connect+REST+Endpoints > > On Wed, Sep 21, 2022 at 4:59 AM Dániel Urbán > wrote: > > > Hi Chris, > > > > About the worker id: makes sense. I changed the wording, but kept it > listed > > as i

Re: Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2022-09-21 Thread Dániel Urbán
rker? > > Finally, do you think we could change "mm.enable.rest" to > "mm.enable.internal.rest"? That way, if we want to introduce a > public-facing REST API later on, we can use "mm.enable.rest" as the name > for that property. > > Cheers, > &

Re: Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2022-09-16 Thread Dániel Urbán
that seems > like > > it'll be affected by switching to an internal-only REST API until those > > changes are published, but should be able to review everything else. > > > > Cheers, > > > > Chris > > > > On Mon, Aug 29, 2022 at 6:57 AM Dániel Urbán >

Re: Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2022-09-16 Thread Dániel Urbán
Hi Chris, I went through the KIP and updated it based on our discussion. I think your suggestions simplified (and shortened) the KIP significantly. Thanks, Daniel Dániel Urbán ezt írta (időpont: 2022. szept. 16., P, 15:15): > Hi Chris, > > 1. For the REST-server-per-flow setup, it m

Re: Transactions, delivery timeout and changing transactional producer behavior

2022-09-09 Thread Dániel Urbán
. júl. 25., H, 15:28): > Hi Luke & Artem, > > We prepared the fix, would you please help in getting a committer-reviewer > to get this issue resolved? > > Thanks, > Viktor > > On Fri, Jul 8, 2022 at 12:57 PM Dániel Urbán > wrote: > > > Submitted a PR with th

Re: [VOTE] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2023-01-11 Thread Dániel Urbán
As for the vote, thanks for participating, the vote passed with 3 binding and 1 non-binding votes. Moving the KIP forward. Daniel Dániel Urbán ezt írta (időpont: 2023. jan. 11., Sze, 13:28): > Hi Mickael, > Yes, after this KIP is implemented, the nested Connect workers inside MM2 > wi

Re: Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2023-01-11 Thread Dániel Urbán
s Egerton > wrote: > > > > Thanks Daniel! No further comments from me, looks good. > > > > On Tue, Sep 27, 2022 at 4:39 AM Dániel Urbán > wrote: > > > > > Hi Chris, > > > > > > I understand your points, and I agree that this path i

Re: [VOTE] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2023-01-11 Thread Dániel Urbán
Hi Mickael, Yes, after this KIP is implemented, the nested Connect workers inside MM2 will behave the same way as vanilla Connect, and EOS will be supported. I would like to ask Chris to confirm, as I'm not too familiar with the details of KIP-618, but I really don't see any issues after KIP-710

Re: [DISCUSS] KIP-918: MM2 Topic And Group Listener

2023-04-18 Thread Dániel Urbán
Hello everyone, I would like to bump this KIP. Please consider reviewing it, as it would improve the monitoring capabilities around MM2. I also submitted a PR (https://github.com/apache/kafka/pull/13595) to demonstrate the current state of the KIP. Thanks in advance, Daniel Dániel Urbán ezt

Re: Re: [DISCUSS] KIP-918: MM2 Topic And Group Listener

2023-04-19 Thread Dániel Urbán
the topic has no data), and > using MirrorCheckpointMetrics.CHECKPOINT_LATENCY can monitor the currently > replicated group list (if it is wrong, please correct me). > > best, > hudeqi > > Dániel Urbán urb.dani...@gmail.com写道: > > Hello everyone, > > > &

Re: [DISCUSS] KIP-918: MM2 Topic And Group Listener

2023-04-19 Thread Dániel Urbán
roup list we > configured, or the topic list and group list that are currently being > replicated? If it is the latter, shouldn't it be detected for a group that > has not committed an offset? I don't know if I understand correctly. > > best, > hudeqi > > > -原始邮件-

Re: [DISCUSS] KIP-918: MM2 Topic And Group Listener

2023-04-21 Thread Dániel Urbán
> Viktor > > On Wed, Apr 19, 2023 at 1:42 PM Dániel Urbán > wrote: > > > I wouldn't really include a non-existent group (same as we don't care > about > > a non-existent topic), that doesn't really matter. > > I think having an existing group which does

[DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-03-30 Thread Dániel Urbán
Hello everyone, I would like to kick off a discussion about KIP-916: https://cwiki.apache.org/confluence/display/KAFKA/KIP-916%3A+MM2+distributed+mode+flow+log+context The KIP aims to enhance the diagnostic information for MM2 distributed mode. MM2 relies on multiple Connect worker instances

[DISCUSS] KIP-918: MM2 Topic And Group Listener

2023-04-13 Thread Dániel Urbán
Hi everyone, I would like to start a discussion on KIP-918: MM2 Topic And Group Listener ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-918%3A+MM2+Topic+And+Group+Listener ). This new feature of MM2 would allow following the latest set of replicated topics and groups, which is currently

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-04-13 Thread Dániel Urbán
Thanks for your comments in advance, Daniel Dániel Urbán ezt írta (időpont: 2023. márc. 30., Cs, 18:24): > Hello everyone, > > I would like to kick off a discussion about KIP-916: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-916%3A+MM2+distributed+mode+flow+log+context >

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

2023-07-07 Thread Dániel Urbán
ill > > be in order of increasing offset for each share-partition" > > > > If the share-consumer uses thread pool internally and acknowledges the > > records in out-of-order fashion. > > Will this use case be supported? The "Managing durable share-partition > &

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

2023-07-10 Thread Dániel Urbán
and the consumer), and > the wire protocol, and I am already worried about the metadata we might > need to track for queuing in general. > > Does this make sense? > > > -Matthias > > > > On 7/7/23 01:35, Dániel Urbán wrote: > > Hi Matthias, > > Can you please

[VOTE] KIP-916: MM2 distributed mode flow log context

2023-07-10 Thread Dániel Urbán
Hello everyone, I would like to start a vote on KIP-916: MM2 distributed mode flow log context. The KIP aims to improve the logging of MM2 distributed mode. It is a relatively small change, but it has a big impact, as the current logs are very hard to decipher. (The current logs are based on the

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

2023-06-02 Thread Dániel Urbán
t state is a bit > more > tricky. > > I think it’s worth thinking ahead to how EOS will work and also another > couple of enhancements (key-based ordering and acquisition lock > extension) so it’s somewhat future-proof. > > Thanks, > Andrew > > > On 1 Jun 2023, at

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-06-05 Thread Dániel Urbán
ot;connector.context" key. > > Thoughts? > > [1] - > > https://github.com/apache/kafka/blob/146a6976aed0d9f90c70b6f21dca8b887cc34e71/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java#L255 > > Cheers, > > Chris > > On Tue, May 23, 2023 at 11:36 AM Dánie

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-06-05 Thread Dániel Urbán
I updated the KIP accordingly. Tried to come up with more generic terms in the Connect code instead of referring to "flow" anywhere. Daniel Dániel Urbán ezt írta (időpont: 2023. jún. 5., H, 14:49): > Hi Chris, > > Thank you for your comments! > > I agree that t

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-06-07 Thread Dániel Urbán
y? > > 5. RE the "Test Plan" section: can you provide a little more detail of > which cases we'll be covering with the proposed unit tests? Will we be > verifying that the MDC context is set in various places? If so, which > places? And the same with thread names? (There does

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

2023-06-07 Thread Dániel Urbán
r the AdminClient can be used to set a > non-default > value for the isolation level for a share group. The value is applied when > the first > member joins. > > Thanks, > Andrew > > > On 2 Jun 2023, at 10:02, Dániel Urbán wrote: > > > > Hi Andrew, >

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

2023-06-07 Thread Dániel Urbán
gt; Hi Daniel, > Kind of. I don’t want a transaction abort to cause disappearance of > records which are already in-flight. A “pending” state doesn’t seem > helpful for read_committed. There’s no such disappearance problem > for read_uncommitted. > > Thanks, > Andrew > >

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-05-23 Thread Dániel Urbán
Hi Chris, Thank you for your comments! 1. This approach is OK for me. I thought that the "sample" configs in the repo do not fall into the same category as the default of a new config. Adding a commented line instead should be ok, and the future opt-out change sounds good to me. 2. Besides the

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-06-12 Thread Dániel Urbán
Updated the KIP with a few example log lines before/after the change. Daniel Dániel Urbán ezt írta (időpont: 2023. jún. 7., Sze, 13:59): > Hi Chris, > > Thank you for your comments! I updated the KIP. I still need to add the > example before/after log lines, will do that soon, but

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

2023-06-01 Thread Dániel Urbán
Hi Andrew, Thank you for the KIP, exciting work you are doing :) I have 2 questions: 1. I understand that EOS won't be supported for share-groups (yet), but read_committed fetch will still be possible, correct? 2. I have a very broad question about the proposed solution: why not let the

Re: [DISCUSS] KIP-918: MM2 Topic And Group Listener

2023-06-28 Thread Dániel Urbán
eplicated" as a prefix > ("replicatedTopics", "replicatedGroups"). > > [1] - > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-558%3A+Track+the+set+of+actively+used+topics+by+connectors+in+Kafka+Connect > [2] - > > https://kafka.apa

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-06-28 Thread Dániel Urbán
If there are no further comments, I will kick off a vote soon for the KIP. Dániel Urbán ezt írta (időpont: 2023. jún. 12., H, 11:27): > Updated the KIP with a few example log lines before/after the change. > Daniel > > Dániel Urbán ezt írta (időpont: 2023. jún. 7., > Sze, 13:59)

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-05-11 Thread Dániel Urbán
Hello everyone, I would like to bump this thread, pretty straightforward KIP, and helps a lot with MM2 dedicated mode diagnostics. Thanks in advance, Daniel Dániel Urbán ezt írta (időpont: 2023. máj. 4., Cs, 14:08): > Hi Viktor, > > Thank you for your comments. I agree that this chan

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-05-04 Thread Dániel Urbán
ogging framework. > > A minor note: please update the JIRA link in the KIP to point to the right > one. > > Best, > Viktor > > On Thu, Apr 13, 2023 at 2:19 PM Dániel Urbán > wrote: > > > Hi everyone, > > > > I would like to bump this thread. I

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

2024-01-08 Thread Dániel Urbán
Hi Elxan, +1 (non-binding) Thanks for the KIP, this will be a very useful metric for MM! Daniel Elxan Eminov ezt írta (időpont: 2024. jan. 7., V, 2:17): > Hi all, > Bumping this for visibility > > On Wed, 3 Jan 2024 at 18:13, Elxan Eminov wrote: > > > Hi All, > > I'd like to initiate a vote