gt; the perfect solution for this either. Introducing yet another interface
> > doesn't seem like what we should do here and since the filter
> > implementations can be reused, it has some benefits.
> > I have no further questions. If we don't see any more responses in
Hi everyone,
Bumping again, maybe someone has an interest in this pretty useful KIP.
Daniel
Dániel Urbán ezt írta (időpont: 2025. febr. 11., K,
10:42):
> Hi,
> Just checking in, maybe someone got interested in the KIP.
> Daniel
>
> Dániel Urbán ezt írta (időpont: 2025. jan
tually trickle down to the protocols. If cluster
> admins
> > > who
> > > > are outside VCs tried to list topics for instance, they would need to
> > be
> > > > able to distinguish between mytopic in VC1 and mytopic in VC2. Right
> > now
> > &g
Hi Greg,
Trying to respond to some of your questions/points:
1. We started out with the exact model you propose here - if VCs can be
nested, the root VC can be a special one, and everything else falls into
place. The issue we encountered here is how can clients inside a VC can
refer to topics lin
Hi,
Just checking in, maybe someone got interested in the KIP.
Daniel
Dániel Urbán ezt írta (időpont: 2025. jan. 6., H,
9:06):
> Hey everyone,
> Trying to bump once more, maybe someone will notice :)
> TIA
> Daniel
>
> Dániel Urbán ezt írta (időpont: 2024. dec. 17.,
&g
Hi,
I'm very late to this discussion, but I saw the part about the performance
impact of fetching the LEOs - I was curious how that was exactly
experienced? Was it using consumer.endOffsets()?
There is also a consumer.currentLag() method, which uses locally-cached LEO
info. It might not be as preci
Hey everyone,
Trying to bump once more, maybe someone will notice :)
TIA
Daniel
Dániel Urbán ezt írta (időpont: 2024. dec. 17., K,
18:26):
> Hi everyone,
> Bumping in hope for some votes - consider checking this, small KIP with
> some useful improvements.
> TIA
> Daniel
>
&
Hi everyone,
Bumping in hope for some votes - consider checking this, small KIP with
some useful improvements.
TIA
Daniel
Dániel Urbán ezt írta (időpont: 2024. dec. 13., P,
9:22):
> Bumping - please consider voting on this KIP.
> TIA
> Daniel
>
> Dániel Urbán ezt írta (időpon
Bumping - please consider voting on this KIP.
TIA
Daniel
Dániel Urbán ezt írta (időpont: 2024. dec. 9., H,
9:06):
> Gentle bump - please consider checking the KIP and voting.
> Daniel
>
> Dániel Urbán ezt írta (időpont: 2024. dec. 5.,
> Cs, 12:08):
>
>> Bumping this
Gentle bump - please consider checking the KIP and voting.
Daniel
Dániel Urbán ezt írta (időpont: 2024. dec. 5., Cs,
12:08):
> Bumping this vote - the change has a relatively small footprint, but fills
> a sizable gap in MM2.
> Please consider checking the KIP and chiming in.
>
>
> On Mon, Dec 2, 2024 at 10:36 AM Vidor Kanalas
> wrote:
>
> > Hi, thanks for the KIP!
> > +1 (non-binding)
> >
> > Best,
> > Vidor
> > On Mon, Dec 2, 2024 at 10:15 AM Dániel Urbán
> > wrote:
> >
> > > Hi everyone,
> >
Hi everyone,
I'd like to start the vote on KIP-1098: Reverse Checkpointing in
MirrorMaker (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1098%3A+Reverse+Checkpointing+in+MirrorMaker
).
TIA,
Daniel
This
> would encapsulate the default behavior with DefaultReplicationPolicy. Also,
> I think it would be nice to provide an implementation for Identity as well
> since we support that replication policy too. What do you think?
>
> Best,
> Viktor
>
> On Wed, Nov 20, 2024 at 12
Hi everyone,
Bumping this - I think this gap affects the lives of many MM2 users, would
be great to fill it. Any comments are welcome.
TIA
Daniel
Dániel Urbán ezt írta (időpont: 2024. nov. 15., P,
13:05):
> Hi Vidor,
>
> KV2 and KV3. I think it is a common setup, but it is usually des
use case, but
> if it is, we need to make sure that CGs can get reverse checkpointed even
> if they initially don’t exist on the cluster. (it’s not a traditional
> failover + failback scenario). This is why I was thinking that a reverse
> checkpointing group filter could be useful, but I ag
an existing filter that has similar
> functionality (filtering topics) and users could be more familiar with it.
>
> What do you think?
>
> Viktor
>
> On Thu, Nov 7, 2024 at 9:21 AM Dániel Urbán wrote:
>
> > Gentle bump - any comments are welcome.
> > This could fill
Gentle bump - any comments are welcome.
This could fill an important gap in MM2, and would be nice to fix.
TIA
Daniel
Dániel Urbán ezt írta (időpont: 2024. nov. 4., H,
11:00):
> Hi Vidor,
>
> Thank you for your comments!
>
> 1. I think the optimization sounds nice, but would not
Hi Manikumar,
Thanks for the KIP! I have a few questions:
1. Why did you choose the "minus-" prefix for the new value format? Can't
we just drop the prefix? If we have to keep the prefix, I'd suggest
"last-N-Duration", as it somewhat "rhymes" with latest-earliest.
2. Should minutes and seconds be
r gets busy at times. Or even have a new consumer group consume
> messages from the passive cluster for a while, before “failing it over” to
> the active cluster. Is this something that you would recommend using the
> feature for?
>
>
>
> Best,
>
> Vidor
>
> On
owing days but I wanted to
> reply to you with what I have so far to keep this going.
>
> Best,
> Viktor
>
> On Fri, Oct 25, 2024 at 5:32 PM Dániel Urbán
> wrote:
>
> > Hi,
> >
> > One more update. As I was working on the PR, I realized that the only way
>
/pull/17593
Daniel
Dániel Urbán ezt írta (időpont: 2024. okt. 24., Cs,
15:22):
> Hi all,
> Just a bump/minor update here:
> As I've started working on a POC of the proposed solution, I've realised
> that the hard requirement related to the ReplicationPolicy implementati
Hi all,
Just a bump/minor update here:
As I've started working on a POC of the proposed solution, I've realised
that the hard requirement related to the ReplicationPolicy implementation
can be eliminated, updated the KIP accordingly.
Daniel
Dániel Urbán ezt írta (időpont: 2024. okt.
ect.
>
> Thanks,
> Mickael
>
> On Mon, Oct 21, 2024 at 2:55 PM Dániel Urbán
> wrote:
> >
> > Hi Viktor,
> >
> > Thank you for the comments!
> >
> > SVV1: I think the feature has some performance implications. If the
> reverse
> > checkpo
re to
> everyone. Of course there can be replication policies like the identity one
> that by design disallows this feature, but for that, see my previous point.
>
> Best,
> Viktor
>
> On Fri, Oct 18, 2024 at 3:30 PM Dániel Urbán
> wrote:
>
> > Hi everyone,
> &
Hi everyone,
I'd like to start the discussion on KIP-1098: Reverse Checkpointing (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1098%3A+Reverse+Checkpointing)
which aims to minimize message reprocessing for consumers in failbacks.
TIA,
Daniel
Somogyi-Vass
> > > wrote:
> > >
> > > > +1 (binding)
> > > >
> > > > Thanks for this improvement. If you have a PR, you can ping me for
> review.
> > > >
> > > > On Fri, Sep 27, 2024 at 4:07 PM Dániel Urbán
Hi everyone,
I would like to start a vote on KIP-1089: Allow disabling heartbeats
replication in MirrorSourceConnector (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1089%3A+Allow+disabling+heartbeats+replication+in+MirrorSourceConnector
).
TIA,
Daniel
Bump - I'll start a vote soon if there are no further comments.
Daniel
Dániel Urbán ezt írta (időpont: 2024. szept. 19.,
Cs, 12:23):
> Bump - I know this is not a complicated proposal, but if you have any
> inputs, please share.
> TIA,
> Daniel
>
> Dániel Urbán ezt írt
Bump - I know this is not a complicated proposal, but if you have any
inputs, please share.
TIA,
Daniel
Dániel Urbán ezt írta (időpont: 2024. szept. 13.,
P, 16:54):
> Hi Viktor,
> Thanks for your comments. Technically, yes, a policy subclass can be used
> to intentionally misconf
, one deployed
> multiple heartbeat connectors with different names and compatible filters?
> If one could specify the heartbeat topic name for only the given connector,
> then it would ignore the others (but maybe there is a mistake in my
> thinking).
>
> Thanks,
> Viktor
>
Hi everyone,
I'd like to kick off the discussion of a small improvement in the
MirrorSourceConnector:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1089%3A+Allow+disabling+heartbeats+replication+in+MirrorSourceConnector
The change would be fairly simple, and enables some more advanced use-c
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 for
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 K
data (for both the broker 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 Matth
r in the ConsumerRecords
> > object, which will
> > 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 suppo
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)
s, or we could possibly use "replicated" 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] -
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
Sze, 17:28):
> 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,
>
he 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,
> > Tha
expect users to find out about the
> new MDC key?
>
> 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
&g
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 the toSt
roduction of the "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, 20
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
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 share-gro
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 MD
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
e idiomatic to the
> logging 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 li
n easier.
>
> Best,
> 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 thin
topic list and group 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
>
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,
> >
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
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 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
>
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 nest
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
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 pa
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
> [1] -
>
> https://cwiki.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
ST API is
> enabled on a worker?
>
> 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 prop
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
I'll hold off on commenting on anything 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
> >
> >
. 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
dd the feature flag in this KIP right now to
> address 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
ode (or some substitute that accomplishes the same goal). I don't
> think this will 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 eve
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/
ward to your 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?
> &g
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
er
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
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 doe
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
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 star
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, which
Hi,
I'd like to access the wiki and contribute to KIPs, please add me. My
username is urbandan.
Thanks in advance,
Daniel
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
mind. Did you have 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.
a PoC with 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 F
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 audi
; argument 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,
>
> P
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
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 o
at 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
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
> +1 (non-binding), 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ánie
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:
>
>
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.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-635%3A+GetOffsetShell%3A+support+fo
tShell tool. I 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
>
oes the 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, suppo
> 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://
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):
>
we need 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
&
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 thin
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
93 matches
Mail list logo