Hi Chris,

Thank you for your comments, and sorry for the late reply.

1. Having a single, connector-reported metric for the topics, and another
one for the groups sounds good to me. The only issue I see here is that I'm
not familiar with any non-primitive metrics in the Kafka codebase, and
don't know if introducing a map/list type metric value will be a problem.
2. The intention is to provide the full set each time. A delta based
approach could be possible, but I think it would be an unnecessary
complication. If we go with a metric instead, we should just stick to the
full set.

I will update the KIP with the metric based approach.

Thanks,
Daniel

Chris Egerton <chr...@aiven.io.invalid> ezt írta (időpont: 2023. jún. 6.,
K, 16:32):

> Hi Daniel,
>
> Thanks for the KIP! I see the value in exposing information on replicated
> topics and groups. For one, it matches a similar feature added to Kafka
> Connect in KIP-558 [1], where we started tracking the set of topics that
> connectors interacted with over their lifetime. And there's also the use
> case you provided about identifying the provenance of topics replicated
> with the identity replication policy (or really, any policy that doesn't
> preserve information about the source cluster). Finally, it seems like a
> decent debugging aid for prototyping and initially standing up MM2
> clusters, and a liveness check for existing ones.
>
> Here are my thoughts so far:
>
> 1. I know that MM2 has a lot of pluggable interfaces already but I'm always
> a little hesitant to introduce another one. One alternative could be to add
> new metrics for the sets of replicated topics and groups. Users can already
> implement pluggable metrics reporters [2], which could be a substitute for
> the listener interface proposed in the KIP.
>
> 2. Is the intention to provide the listener with the total current set of
> replicated groups and topics every time that set is computed? Or is the
> listener given the complete set the first time and a delta other times?
> Based on the type signatures of the interface methods I'm guessing it's the
> former, but based on the names (which use the word "changed") it seems like
> the latter. If we use complete sets, I think "refreshed" or "computed" may
> be better as suffixes, 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] -
>
> https://kafka.apache.org/34/javadoc/org/apache/kafka/common/metrics/MetricsReporter.html
>
> Cheers,
>
> Chris
>
> On Fri, Apr 21, 2023 at 9:00 AM Dániel Urbán <urb.dani...@gmail.com>
> wrote:
>
> > Thanks for the comments Viktor.
> > 1. My original motivation was IdentityReplicationPolicy based monitoring.
> > The current MirrorClient implementation cannot list the replica topics of
> > the target cluster. I think relying on the topic-partition level metrics
> is
> > a complex solution. Instead, I would like to make it simple to collect
> all
> > the replicated topics of a flow, without relying on the name of the
> topics.
> > Then, I simply tried to generalize the approach.
> > 2. Checkpoint metrics are reported per (group, topic, partition), it
> means
> > that there is no metric associated with a group. If a filter picks up a
> > group, but the group doesn't have committed offsets for any of the
> > replicated partitions, there is no metric to be eagerly registered. This
> is
> > a difference between how topic replication and group checkpointing works
> -
> > empty topics are still picked up for partition creation and to consume
> from
> > them. Groups are only picked up if they have committed offsets already.
> > 3. Not exactly sure what is the added value of listing all
> > topic-partitions, but that information is available where the filtering
> > happens. For groups, we don't have anything else besides the group name,
> so
> > we cannot really provide more info at that point without significantly
> > changing the refresh group logic.
> >
> > Thanks,
> > Daniel
> >
> > Viktor Somogyi-Vass <viktor.somo...@cloudera.com.invalid> ezt írta
> > (időpont: 2023. ápr. 21., P, 11:43):
> >
> > > Hi all,
> > >
> > > A couple of comments:
> > > 1) Regarding the motivation: is the motivation simply monitoring
> related
> > or
> > > are there any other reasons to this?
> > > 2) Can we change monitoring to be identical to filters, so that what is
> > > actively filtered, we monitor exactly those topics and groups? (So
> group
> > > metrics aren't added lazily when a checkpoint is created but when the
> > > filter is changed.)
> > > 3) Not sure if we want to widen the scope but since these are
> interfaces
> > > I'd use TopicPartition and some kind of GroupDescription classes (not
> > sure
> > > if the latter exists) instead of Strings. If later on we'll need extra
> > > properties for these then it can be added on easier.
> > >
> > > Best,
> > > Viktor
> > >
> > > On Wed, Apr 19, 2023 at 1:42 PM Dániel Urbán <urb.dani...@gmail.com>
> > > 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 doesn't have an offset to
> > > checkpoint
> > > > is equivalent to a topic having no records to replicate from the
> > > monitoring
> > > > perspective.
> > > >
> > > > I think the precise way to put it is to monitor the topics and groups
> > > > picked up by the filtering logic of MM2. "The list currently
> > replicated"
> > > is
> > > > not a good definition, as an empty topic would still be interesting
> for
> > > > monitoring purposes, even if there is no message to replicate.
> > > > I think the core motivation is to capture the output of the
> > > > TopicFilter/GroupFilter + the extra, built-in logic of MM2 related to
> > > > filtering (e.g. internal topics are never replicated, the heartbeats
> > > topics
> > > > are always replicated, and so on). This logic is too complex to
> > reproduce
> > > > in an external monitoring system, as it would need to use the exact
> > same
> > > > TopicFilter/GroupFilter configs as MM2 is using, and then implement
> the
> > > > additional built-in logic of MM2 to finally get the topics and groups
> > > > picked up by the replication.
> > > >
> > > > I think this would be useful in any replication setups (finding the
> > > > effective list of filtered topics and groups), but especially useful
> > when
> > > > using the IdentityReplicationPolicy. One gap related to the identity
> > > policy
> > > > is that we cannot find the replica topics of a specific flow, even
> when
> > > > using MirrorClient, or having access to the source and target Kafka
> > > > clusters, as the "traditional" way of finding replica topics is based
> > on
> > > > topic naming and the ReplicationPolicy.
> > > >
> > > > Thanks,
> > > > Daniel
> > > >
> > > > hudeqi <16120...@bjtu.edu.cn> ezt írta (időpont: 2023. ápr. 19.,
> Sze,
> > > > 10:58):
> > > >
> > > > > Thanks for your reply, Daniel.
> > > > > Regarding the group list, do you mean that if the group of the
> source
> > > > > cluster has not committed an offset (the group does not exist or
> the
> > > > group
> > > > > has not committed an offset to the topic being replicated), then
> the
> > > > > current metric cannot be collected? Then this involves the question
> > of
> > > > > motivation: Do we want to monitor the 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
> > > > >
> > > > >
> > > > > &gt; -----原始邮件-----
> > > > > &gt; 发件人: "Dániel Urbán" <urb.dani...@gmail.com>
> > > > > &gt; 发送时间: 2023-04-19 15:50:01 (星期三)
> > > > > &gt; 收件人: dev@kafka.apache.org
> > > > > &gt; 抄送:
> > > > > &gt; 主题: Re: Re: [DISCUSS] KIP-918: MM2 Topic And Group Listener
> > > > > &gt;
> > > > > </urb.dani...@gmail.com>
> > > >
> > >
> >
>

Reply via email to