Re: Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-17 Thread Viktor Somogyi-Vass
Hi Greg,

> I don't think I understand what you mean here. Are you suggesting an
alternative to the Admin API? An external project could certainly
build such a component with the Admin API.

So I was thinking of something more complex than the Admin API, an external
service that can instruct clusters (using the Admin API) to start or stop
replication for certain partitions, list their replication flows, throttle
them and so on. With this you won't be able to control just one cluster but
you could register your clusters and control them in a centralized fashion.
Similar to what RestServer is to Connect, though with this you don't create
connectors but replication flows.

I've updated the KIP with my remote log storage thoughts (in data
semantics).

Best,
Viktor

On Sat, Oct 7, 2023 at 7:22 PM Greg Harris 
wrote:

> Hello hudeqi,
>
> I apologize if the KIP and discussion have diverged, as I've been
> trying to add detail rather than propose changes.
>
> > Why can't we use the follower fetch protocol?
>
> What you've described sounds like a very reasonable implementation of
> CCR. I purposely have not specified any implementation details so far
> and have been focusing only on the user-facing semantics of the
> feature. You are of course welcome to add details of how the
> follower-fetch implementation would work to the KIP.
>
> I think maybe this wording in the KIP was ambiguous: "Are not eligible
> for fetch-from-follower on the source cluster". I clarified the
> justification for this earlier in Tom's point D3. But to make the
> statement itself more clear:
>
> Consumers on the source cluster will not see a cross-cluster leader or
> followers as valid replicas to fetch from for load-sharing or latency
> optimization. Consumers on the target cluster will see the
> cross-cluster followers as valid replicas to fetch from, as they will
> appear as normal replicas. This was only meant to describe the
> relationship of the remote replicas with Consumers, not with each
> other.
>
> I hope this is more clear.
> Greg
>
> On Sat, Oct 7, 2023 at 1:38 AM hudeqi <16120...@bjtu.edu.cn> wrote:
> >
> > Hi, Greg.
> >
> > After reading this KIP and your discussion, I feel that it is very
> divergent. I can only start from one of them:
> > Why can't we use the follower fetch protocol? The leader of the target
> cluster topic partition can be treated as the follower of the source
> cluster topic partition leader, and the fetched data is directly appended
> to the local log (the remote fetch thread is inherited to the follower
> fetch thread, thereby retaining the offset of the log), so that consumer/
> producer client can be omitted. Of course, this is just data replication. I
> may have to think more about group offset/acl/config replication.
> >
> > best,
> > hudeqi
>


Re: Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-07 Thread Greg Harris
Hello hudeqi,

I apologize if the KIP and discussion have diverged, as I've been
trying to add detail rather than propose changes.

> Why can't we use the follower fetch protocol?

What you've described sounds like a very reasonable implementation of
CCR. I purposely have not specified any implementation details so far
and have been focusing only on the user-facing semantics of the
feature. You are of course welcome to add details of how the
follower-fetch implementation would work to the KIP.

I think maybe this wording in the KIP was ambiguous: "Are not eligible
for fetch-from-follower on the source cluster". I clarified the
justification for this earlier in Tom's point D3. But to make the
statement itself more clear:

Consumers on the source cluster will not see a cross-cluster leader or
followers as valid replicas to fetch from for load-sharing or latency
optimization. Consumers on the target cluster will see the
cross-cluster followers as valid replicas to fetch from, as they will
appear as normal replicas. This was only meant to describe the
relationship of the remote replicas with Consumers, not with each
other.

I hope this is more clear.
Greg

On Sat, Oct 7, 2023 at 1:38 AM hudeqi <16120...@bjtu.edu.cn> wrote:
>
> Hi, Greg.
>
> After reading this KIP and your discussion, I feel that it is very divergent. 
> I can only start from one of them:
> Why can't we use the follower fetch protocol? The leader of the target 
> cluster topic partition can be treated as the follower of the source cluster 
> topic partition leader, and the fetched data is directly appended to the 
> local log (the remote fetch thread is inherited to the follower fetch thread, 
> thereby retaining the offset of the log), so that consumer/ producer client 
> can be omitted. Of course, this is just data replication. I may have to think 
> more about group offset/acl/config replication.
>
> best,
> hudeqi


Re: Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-07 Thread hudeqi
Hi, Greg.

After reading this KIP and your discussion, I feel that it is very divergent. I 
can only start from one of them:
Why can't we use the follower fetch protocol? The leader of the target cluster 
topic partition can be treated as the follower of the source cluster topic 
partition leader, and the fetched data is directly appended to the local log 
(the remote fetch thread is inherited to the follower fetch thread, thereby 
retaining the offset of the log), so that consumer/ producer client can be 
omitted. Of course, this is just data replication. I may have to think more 
about group offset/acl/config replication.

best,
hudeqi

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-05 Thread Greg Harris
Hey Viktor,

Please do! This is a draft, and it's open for you to edit to include
your new ideas :)

I don't think I understand what you mean here. Are you suggesting an
alternative to the Admin API? An external project could certainly
build such a component with the Admin API.

Thanks, Greg

On Thu, Oct 5, 2023 at 6:33 AM Viktor Somogyi-Vass
 wrote:
>
> Hi Greg,
>
> Sure, I'll expand it with my thoughts. Is it fine if I add it to the KIP
> and update this discussion?
>
> Another thing that crossed my mind is that in MM2 you can handle configs
> and replication flow in a central place because it is a separate component.
> I think that for use-cases where there are many replication flows, this
> aspect can be useful (as Kafka itself is useful for microservices). For CCR
> too it could be useful to have some kind of separated service that collects
> this information. It could also serve as an admin endpoint (swagger maybe?)
> for managing flows and configuration. With this you could instruct clusters
> to create/pause/delete replications. What do you think?
>
> Thanks,
> Viktor
>
>
>
> On Wed, Oct 4, 2023 at 6:20 PM Greg Harris 
> wrote:
>
> > Hey Viktor,
> >
> > Thanks for thinking about Tiered Storage. I'm not so familiar there,
> > so if you could add some of your expectations about how the two
> > features will interact, I would appreciate that.
> >
> > It appears to me that follower-fetch-from-remote is a significant
> > optimization within TS, and so similar optimizations to support
> > cross-cluster-replicate-from-remote and out-of-band remote replication
> > could also be desirable.
> > I think we can explore the idea further, and make sure that CCR is
> > extensible to tiered topics if it doesn't make it into the initial
> > implementation.
> >
> > Thanks!
> > Greg
> >
> > On Wed, Oct 4, 2023 at 6:13 AM Viktor Somogyi-Vass
> >  wrote:
> > >
> > > Hi Greg,
> > >
> > > Thanks for the answers. I think they all make sense.
> > >
> > > Another point I realized last evening is that now that tiered storage
> > (TS)
> > > is available, it might complicate things with CCR. What I'm thinking of
> > is
> > > that if you have multiple clusters in multiple regions, enabling the
> > object
> > > storage's replication between zones could be much more cost efficient
> > than
> > > replicating local+remote offsets through Kafka. You'd only need to copy
> > > local segments over and remote partition replication would be done by the
> > > remote layer. Or the user could simply choose to not replicate remote
> > > segments between regions but instead just reference them (so that the
> > > backup cluster's remote offsets point to the original region). These
> > > options however likely require bigger coordination between clusters than
> > in
> > > pre-TS Kafka. Do you think we should take this into consideration in the
> > > design and in the UX?
> > >
> > > Thanks,
> > > Viktor
> > >
> > > On Tue, Oct 3, 2023 at 6:30 PM Greg Harris  > >
> > > wrote:
> > >
> > > > Hi Viktor,
> > > >
> > > > Thanks for your questions! I agree, replication is very fundamental in
> > > > Kafka, so it's been implemented in many different ways by different
> > > > people. I hope that this is the last implementation we'll need, but
> > > > every software engineer says that :)
> > > >
> > > > GT-1: I think as this KIP is very focused on the UX of the feature,
> > > > that user stories are appropriate to include. I think it isn't
> > > > necessary to explain how the different applications are accomplished
> > > > with MM2 or other solutions, but describing what they will look like
> > > > after this KIP would be a wonderful addition. +1
> > > >
> > > > MM2-1: I think that replacing the consumer is insufficient, as we need
> > > > a more expressive producer as well. This is not possible within the
> > > > design constraints of MM2 as a Connector, as MM2 uses the
> > > > connect-managed producer. This could be implemented in MM3 as a new
> > > > process that can use more expressive "internal clients", but then
> > > > we've thrown away the Connect runtime that made MM2 easier to run for
> > > > some users.
> > > > MM2-2: This is technically possible, but sounds operationally
> > hazardous to
> > > > me.
> > > > MM2-3: From the user perspective, I believe that CCR can be made more
> > > > simple to use and operate than MM2, while providing better guarantees.
> > > > From the implementation standpoint, I think that CCR will be
> > > > significantly more complex, as the architecture of MM2 leverages a lot
> > > > of the Connect infrastructure.
> > > >
> > > > LaK-1: Yes, I think you understand what I was going for.
> > > > LaK-2: I don't think that this is a user experience that we could add
> > > > to CCR without changing the Kafka clients to be aware of both clusters
> > > > concurrently. In order to redirect clients away from a failed cluster
> > > > with a metadata refresh, the cluster that they're currently connected
> > > > to must give them 

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-05 Thread Viktor Somogyi-Vass
Hi Greg,

Sure, I'll expand it with my thoughts. Is it fine if I add it to the KIP
and update this discussion?

Another thing that crossed my mind is that in MM2 you can handle configs
and replication flow in a central place because it is a separate component.
I think that for use-cases where there are many replication flows, this
aspect can be useful (as Kafka itself is useful for microservices). For CCR
too it could be useful to have some kind of separated service that collects
this information. It could also serve as an admin endpoint (swagger maybe?)
for managing flows and configuration. With this you could instruct clusters
to create/pause/delete replications. What do you think?

Thanks,
Viktor



On Wed, Oct 4, 2023 at 6:20 PM Greg Harris 
wrote:

> Hey Viktor,
>
> Thanks for thinking about Tiered Storage. I'm not so familiar there,
> so if you could add some of your expectations about how the two
> features will interact, I would appreciate that.
>
> It appears to me that follower-fetch-from-remote is a significant
> optimization within TS, and so similar optimizations to support
> cross-cluster-replicate-from-remote and out-of-band remote replication
> could also be desirable.
> I think we can explore the idea further, and make sure that CCR is
> extensible to tiered topics if it doesn't make it into the initial
> implementation.
>
> Thanks!
> Greg
>
> On Wed, Oct 4, 2023 at 6:13 AM Viktor Somogyi-Vass
>  wrote:
> >
> > Hi Greg,
> >
> > Thanks for the answers. I think they all make sense.
> >
> > Another point I realized last evening is that now that tiered storage
> (TS)
> > is available, it might complicate things with CCR. What I'm thinking of
> is
> > that if you have multiple clusters in multiple regions, enabling the
> object
> > storage's replication between zones could be much more cost efficient
> than
> > replicating local+remote offsets through Kafka. You'd only need to copy
> > local segments over and remote partition replication would be done by the
> > remote layer. Or the user could simply choose to not replicate remote
> > segments between regions but instead just reference them (so that the
> > backup cluster's remote offsets point to the original region). These
> > options however likely require bigger coordination between clusters than
> in
> > pre-TS Kafka. Do you think we should take this into consideration in the
> > design and in the UX?
> >
> > Thanks,
> > Viktor
> >
> > On Tue, Oct 3, 2023 at 6:30 PM Greg Harris  >
> > wrote:
> >
> > > Hi Viktor,
> > >
> > > Thanks for your questions! I agree, replication is very fundamental in
> > > Kafka, so it's been implemented in many different ways by different
> > > people. I hope that this is the last implementation we'll need, but
> > > every software engineer says that :)
> > >
> > > GT-1: I think as this KIP is very focused on the UX of the feature,
> > > that user stories are appropriate to include. I think it isn't
> > > necessary to explain how the different applications are accomplished
> > > with MM2 or other solutions, but describing what they will look like
> > > after this KIP would be a wonderful addition. +1
> > >
> > > MM2-1: I think that replacing the consumer is insufficient, as we need
> > > a more expressive producer as well. This is not possible within the
> > > design constraints of MM2 as a Connector, as MM2 uses the
> > > connect-managed producer. This could be implemented in MM3 as a new
> > > process that can use more expressive "internal clients", but then
> > > we've thrown away the Connect runtime that made MM2 easier to run for
> > > some users.
> > > MM2-2: This is technically possible, but sounds operationally
> hazardous to
> > > me.
> > > MM2-3: From the user perspective, I believe that CCR can be made more
> > > simple to use and operate than MM2, while providing better guarantees.
> > > From the implementation standpoint, I think that CCR will be
> > > significantly more complex, as the architecture of MM2 leverages a lot
> > > of the Connect infrastructure.
> > >
> > > LaK-1: Yes, I think you understand what I was going for.
> > > LaK-2: I don't think that this is a user experience that we could add
> > > to CCR without changing the Kafka clients to be aware of both clusters
> > > concurrently. In order to redirect clients away from a failed cluster
> > > with a metadata refresh, the cluster that they're currently connected
> > > to must give them that data. But because the cluster failed, that
> > > refresh will not be reliable. With a proxy between the client and
> > > Kafka, that proxy can be available while the original Kafka cluster is
> > > not. Failovers would happen between distinct sets of clients that are
> > > part of the same logical application.
> > >
> > > Thanks for taking a look at the rejected alternatives!
> > > Greg
> > >
> > > On Tue, Oct 3, 2023 at 3:24 AM Viktor Somogyi-Vass
> > >  wrote:
> > > >
> > > > Hi Greg,
> > > >
> > > > Seems like finding the perfect replication 

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-04 Thread Greg Harris
Hey Viktor,

Thanks for thinking about Tiered Storage. I'm not so familiar there,
so if you could add some of your expectations about how the two
features will interact, I would appreciate that.

It appears to me that follower-fetch-from-remote is a significant
optimization within TS, and so similar optimizations to support
cross-cluster-replicate-from-remote and out-of-band remote replication
could also be desirable.
I think we can explore the idea further, and make sure that CCR is
extensible to tiered topics if it doesn't make it into the initial
implementation.

Thanks!
Greg

On Wed, Oct 4, 2023 at 6:13 AM Viktor Somogyi-Vass
 wrote:
>
> Hi Greg,
>
> Thanks for the answers. I think they all make sense.
>
> Another point I realized last evening is that now that tiered storage (TS)
> is available, it might complicate things with CCR. What I'm thinking of is
> that if you have multiple clusters in multiple regions, enabling the object
> storage's replication between zones could be much more cost efficient than
> replicating local+remote offsets through Kafka. You'd only need to copy
> local segments over and remote partition replication would be done by the
> remote layer. Or the user could simply choose to not replicate remote
> segments between regions but instead just reference them (so that the
> backup cluster's remote offsets point to the original region). These
> options however likely require bigger coordination between clusters than in
> pre-TS Kafka. Do you think we should take this into consideration in the
> design and in the UX?
>
> Thanks,
> Viktor
>
> On Tue, Oct 3, 2023 at 6:30 PM Greg Harris 
> wrote:
>
> > Hi Viktor,
> >
> > Thanks for your questions! I agree, replication is very fundamental in
> > Kafka, so it's been implemented in many different ways by different
> > people. I hope that this is the last implementation we'll need, but
> > every software engineer says that :)
> >
> > GT-1: I think as this KIP is very focused on the UX of the feature,
> > that user stories are appropriate to include. I think it isn't
> > necessary to explain how the different applications are accomplished
> > with MM2 or other solutions, but describing what they will look like
> > after this KIP would be a wonderful addition. +1
> >
> > MM2-1: I think that replacing the consumer is insufficient, as we need
> > a more expressive producer as well. This is not possible within the
> > design constraints of MM2 as a Connector, as MM2 uses the
> > connect-managed producer. This could be implemented in MM3 as a new
> > process that can use more expressive "internal clients", but then
> > we've thrown away the Connect runtime that made MM2 easier to run for
> > some users.
> > MM2-2: This is technically possible, but sounds operationally hazardous to
> > me.
> > MM2-3: From the user perspective, I believe that CCR can be made more
> > simple to use and operate than MM2, while providing better guarantees.
> > From the implementation standpoint, I think that CCR will be
> > significantly more complex, as the architecture of MM2 leverages a lot
> > of the Connect infrastructure.
> >
> > LaK-1: Yes, I think you understand what I was going for.
> > LaK-2: I don't think that this is a user experience that we could add
> > to CCR without changing the Kafka clients to be aware of both clusters
> > concurrently. In order to redirect clients away from a failed cluster
> > with a metadata refresh, the cluster that they're currently connected
> > to must give them that data. But because the cluster failed, that
> > refresh will not be reliable. With a proxy between the client and
> > Kafka, that proxy can be available while the original Kafka cluster is
> > not. Failovers would happen between distinct sets of clients that are
> > part of the same logical application.
> >
> > Thanks for taking a look at the rejected alternatives!
> > Greg
> >
> > On Tue, Oct 3, 2023 at 3:24 AM Viktor Somogyi-Vass
> >  wrote:
> > >
> > > Hi Greg,
> > >
> > > Seems like finding the perfect replication solution is a never ending
> > story
> > > for Kafka :).
> > >
> > > Some general thoughts:
> > > GT-1. While as you say it would be good to have some kind of built-in
> > > replication in Kafka, we definitely need to understand the problem better
> > > to provide a better solution. Replication has lots of user stories as you
> > > iterated over a few and I think it's very well worth the time to detail
> > > each one in the KIP. This may help understanding the problem on a deeper
> > > level to others who may want to contribute, somewhat sets the scope and
> > > describes the problem in a way that a good solution can be deduced from
> > it.
> > >
> > > I also have a few questions regarding some of the rejected solutions:
> > >
> > > MM2:
> > > I think your points about MM2 are fair (offset transparency and
> > operational
> > > complexity), however I think it needs more reasoning about why are we
> > > moving in a different direction?
> > > A few 

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-04 Thread Viktor Somogyi-Vass
Hi Greg,

Thanks for the answers. I think they all make sense.

Another point I realized last evening is that now that tiered storage (TS)
is available, it might complicate things with CCR. What I'm thinking of is
that if you have multiple clusters in multiple regions, enabling the object
storage's replication between zones could be much more cost efficient than
replicating local+remote offsets through Kafka. You'd only need to copy
local segments over and remote partition replication would be done by the
remote layer. Or the user could simply choose to not replicate remote
segments between regions but instead just reference them (so that the
backup cluster's remote offsets point to the original region). These
options however likely require bigger coordination between clusters than in
pre-TS Kafka. Do you think we should take this into consideration in the
design and in the UX?

Thanks,
Viktor

On Tue, Oct 3, 2023 at 6:30 PM Greg Harris 
wrote:

> Hi Viktor,
>
> Thanks for your questions! I agree, replication is very fundamental in
> Kafka, so it's been implemented in many different ways by different
> people. I hope that this is the last implementation we'll need, but
> every software engineer says that :)
>
> GT-1: I think as this KIP is very focused on the UX of the feature,
> that user stories are appropriate to include. I think it isn't
> necessary to explain how the different applications are accomplished
> with MM2 or other solutions, but describing what they will look like
> after this KIP would be a wonderful addition. +1
>
> MM2-1: I think that replacing the consumer is insufficient, as we need
> a more expressive producer as well. This is not possible within the
> design constraints of MM2 as a Connector, as MM2 uses the
> connect-managed producer. This could be implemented in MM3 as a new
> process that can use more expressive "internal clients", but then
> we've thrown away the Connect runtime that made MM2 easier to run for
> some users.
> MM2-2: This is technically possible, but sounds operationally hazardous to
> me.
> MM2-3: From the user perspective, I believe that CCR can be made more
> simple to use and operate than MM2, while providing better guarantees.
> From the implementation standpoint, I think that CCR will be
> significantly more complex, as the architecture of MM2 leverages a lot
> of the Connect infrastructure.
>
> LaK-1: Yes, I think you understand what I was going for.
> LaK-2: I don't think that this is a user experience that we could add
> to CCR without changing the Kafka clients to be aware of both clusters
> concurrently. In order to redirect clients away from a failed cluster
> with a metadata refresh, the cluster that they're currently connected
> to must give them that data. But because the cluster failed, that
> refresh will not be reliable. With a proxy between the client and
> Kafka, that proxy can be available while the original Kafka cluster is
> not. Failovers would happen between distinct sets of clients that are
> part of the same logical application.
>
> Thanks for taking a look at the rejected alternatives!
> Greg
>
> On Tue, Oct 3, 2023 at 3:24 AM Viktor Somogyi-Vass
>  wrote:
> >
> > Hi Greg,
> >
> > Seems like finding the perfect replication solution is a never ending
> story
> > for Kafka :).
> >
> > Some general thoughts:
> > GT-1. While as you say it would be good to have some kind of built-in
> > replication in Kafka, we definitely need to understand the problem better
> > to provide a better solution. Replication has lots of user stories as you
> > iterated over a few and I think it's very well worth the time to detail
> > each one in the KIP. This may help understanding the problem on a deeper
> > level to others who may want to contribute, somewhat sets the scope and
> > describes the problem in a way that a good solution can be deduced from
> it.
> >
> > I also have a few questions regarding some of the rejected solutions:
> >
> > MM2:
> > I think your points about MM2 are fair (offset transparency and
> operational
> > complexity), however I think it needs more reasoning about why are we
> > moving in a different direction?
> > A few points I can think about what we could improve in MM2 that'd
> > transform it into more like a solution that you aim for:
> > MM2-1. What if we consider replacing the client based mechanism with a
> > follower fetch protocol?
> > MM2-2. Operating an MM2 cluster might be familiar to those who operate
> > Connect anyway. For those who don't, can we provide a "built-in" version
> > that runs in the same process as Kafka, like an embedded dedicated MM2
> > cluster?
> > MM2-3. Will we actually be able to achieve less complexity with a
> built-in
> > solution?
> >
> > Layer above Kafka:
> > LaK-1. Would you please add more details about this? What I can currently
> > think of is that this "layer above Kafka" would be some kind of a proxy
> > which would proactively send an incoming request to multiple clusters
> like
> > 

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-03 Thread Greg Harris
Hi Viktor,

Thanks for your questions! I agree, replication is very fundamental in
Kafka, so it's been implemented in many different ways by different
people. I hope that this is the last implementation we'll need, but
every software engineer says that :)

GT-1: I think as this KIP is very focused on the UX of the feature,
that user stories are appropriate to include. I think it isn't
necessary to explain how the different applications are accomplished
with MM2 or other solutions, but describing what they will look like
after this KIP would be a wonderful addition. +1

MM2-1: I think that replacing the consumer is insufficient, as we need
a more expressive producer as well. This is not possible within the
design constraints of MM2 as a Connector, as MM2 uses the
connect-managed producer. This could be implemented in MM3 as a new
process that can use more expressive "internal clients", but then
we've thrown away the Connect runtime that made MM2 easier to run for
some users.
MM2-2: This is technically possible, but sounds operationally hazardous to me.
MM2-3: From the user perspective, I believe that CCR can be made more
simple to use and operate than MM2, while providing better guarantees.
>From the implementation standpoint, I think that CCR will be
significantly more complex, as the architecture of MM2 leverages a lot
of the Connect infrastructure.

LaK-1: Yes, I think you understand what I was going for.
LaK-2: I don't think that this is a user experience that we could add
to CCR without changing the Kafka clients to be aware of both clusters
concurrently. In order to redirect clients away from a failed cluster
with a metadata refresh, the cluster that they're currently connected
to must give them that data. But because the cluster failed, that
refresh will not be reliable. With a proxy between the client and
Kafka, that proxy can be available while the original Kafka cluster is
not. Failovers would happen between distinct sets of clients that are
part of the same logical application.

Thanks for taking a look at the rejected alternatives!
Greg

On Tue, Oct 3, 2023 at 3:24 AM Viktor Somogyi-Vass
 wrote:
>
> Hi Greg,
>
> Seems like finding the perfect replication solution is a never ending story
> for Kafka :).
>
> Some general thoughts:
> GT-1. While as you say it would be good to have some kind of built-in
> replication in Kafka, we definitely need to understand the problem better
> to provide a better solution. Replication has lots of user stories as you
> iterated over a few and I think it's very well worth the time to detail
> each one in the KIP. This may help understanding the problem on a deeper
> level to others who may want to contribute, somewhat sets the scope and
> describes the problem in a way that a good solution can be deduced from it.
>
> I also have a few questions regarding some of the rejected solutions:
>
> MM2:
> I think your points about MM2 are fair (offset transparency and operational
> complexity), however I think it needs more reasoning about why are we
> moving in a different direction?
> A few points I can think about what we could improve in MM2 that'd
> transform it into more like a solution that you aim for:
> MM2-1. What if we consider replacing the client based mechanism with a
> follower fetch protocol?
> MM2-2. Operating an MM2 cluster might be familiar to those who operate
> Connect anyway. For those who don't, can we provide a "built-in" version
> that runs in the same process as Kafka, like an embedded dedicated MM2
> cluster?
> MM2-3. Will we actually be able to achieve less complexity with a built-in
> solution?
>
> Layer above Kafka:
> LaK-1. Would you please add more details about this? What I can currently
> think of is that this "layer above Kafka" would be some kind of a proxy
> which would proactively send an incoming request to multiple clusters like
> "broadcast" it. Is that a correct assumption?
> LaK-2. In case of a cluster failover a client needs to change bootstrap
> servers to a different cluster. A layer above Kafka or a proxy can solve
> this by abstracting away the cluster itself. It could force out a metadata
> refresh and from that point on clients can fetch from the other cluster. Is
> this problem within the scope of this KIP or not?
>
> Thanks,
> Viktor
>
>
> On Tue, Oct 3, 2023 at 2:55 AM Greg Harris 
> wrote:
>
> > Hey Tom,
> >
> > Thanks for the high-level questions, as I am certainly approaching
> > this KIP differently than I've seen before.
> >
> > I think that ideally this KIP will expand to include lots of
> > requirements and possible implementations, and that through discussion
> > we can narrow the scope and form a roadmap for implementation across
> > multiple KIPs. I don't plan to be the decision-maker for this project,
> > as I'm more interested in building consensus among the co-authors. I
> > can certainly poll that consensus and update the KIP to keep the
> > project moving, and any other co-author can do the same. And to 

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-03 Thread Viktor Somogyi-Vass
Hi Greg,

Seems like finding the perfect replication solution is a never ending story
for Kafka :).

Some general thoughts:
GT-1. While as you say it would be good to have some kind of built-in
replication in Kafka, we definitely need to understand the problem better
to provide a better solution. Replication has lots of user stories as you
iterated over a few and I think it's very well worth the time to detail
each one in the KIP. This may help understanding the problem on a deeper
level to others who may want to contribute, somewhat sets the scope and
describes the problem in a way that a good solution can be deduced from it.

I also have a few questions regarding some of the rejected solutions:

MM2:
I think your points about MM2 are fair (offset transparency and operational
complexity), however I think it needs more reasoning about why are we
moving in a different direction?
A few points I can think about what we could improve in MM2 that'd
transform it into more like a solution that you aim for:
MM2-1. What if we consider replacing the client based mechanism with a
follower fetch protocol?
MM2-2. Operating an MM2 cluster might be familiar to those who operate
Connect anyway. For those who don't, can we provide a "built-in" version
that runs in the same process as Kafka, like an embedded dedicated MM2
cluster?
MM2-3. Will we actually be able to achieve less complexity with a built-in
solution?

Layer above Kafka:
LaK-1. Would you please add more details about this? What I can currently
think of is that this "layer above Kafka" would be some kind of a proxy
which would proactively send an incoming request to multiple clusters like
"broadcast" it. Is that a correct assumption?
LaK-2. In case of a cluster failover a client needs to change bootstrap
servers to a different cluster. A layer above Kafka or a proxy can solve
this by abstracting away the cluster itself. It could force out a metadata
refresh and from that point on clients can fetch from the other cluster. Is
this problem within the scope of this KIP or not?

Thanks,
Viktor


On Tue, Oct 3, 2023 at 2:55 AM Greg Harris 
wrote:

> Hey Tom,
>
> Thanks for the high-level questions, as I am certainly approaching
> this KIP differently than I've seen before.
>
> I think that ideally this KIP will expand to include lots of
> requirements and possible implementations, and that through discussion
> we can narrow the scope and form a roadmap for implementation across
> multiple KIPs. I don't plan to be the decision-maker for this project,
> as I'm more interested in building consensus among the co-authors. I
> can certainly poll that consensus and update the KIP to keep the
> project moving, and any other co-author can do the same. And to set an
> example, I'll clarify your questions and for anything that I agree
> with, I'll ask that you make the update to the KIP, so that the KIP
> captures your understanding of the problem and your requirements. If
> you don't get the chance to make the changes yourself, I'll make sure
> they get included eventually, as they're very good ideas :)
>
> For your remaining questions:
>
> M1: I was trying to draw analogies to databases, but your suggested
> properties are much more compelling and informative. I'd love it if
> you added some formalism here, so that we have a better grasp on what
> we're trying to accomplish. +1
> M2: I think the "asynchronous" problem corresponds to the goal of
> "exactly once semantics" but the two are not obviously opposites. I
> think the MM2 deficiencies could focus less on the architecture
> (asynchronicity) and more on the user-facing effect (semantics). +1
> M3: I had a "non-goals" section that ended up becoming the "rejected
> alternatives" section instead. If you have some non-goals in mind,
> please add them.
> M4+M5: I think it's too early to nail down the assumptions directly,
> but if you believe that "separate operators of source and target" is a
> requirement, that would be good to write down. +1
> M6: That is a concerning edge case, and I don't know how to handle it.
> I was imagining that there would be a many:many relationship of
> clusters and links, but I understand that the book-keeping of that
> decision may be significant.
> M7: I think this may be appropriate to cover in a "user story" or
> "example usages". I naturally thought that the feature would describe
> some minimal way of linking two topics, and the applications
> (combining multiple links, performing failovers, or running
> active-active, etc) would be left to users to define. I included the
> regex configurations because I imagine that creating 100s or 1000s of
> links would be unnecessarily tedious. The feature may also encode
> those use-cases directly as first-class citizens as well.
>
> U1: These are states that can happen in reality, and I meant for that
> section to imply that we should expect these states and model them for
> operations and observability.
>
> D1: I think I may have introduced this 

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-02 Thread Greg Harris
Hey Tom,

Thanks for the high-level questions, as I am certainly approaching
this KIP differently than I've seen before.

I think that ideally this KIP will expand to include lots of
requirements and possible implementations, and that through discussion
we can narrow the scope and form a roadmap for implementation across
multiple KIPs. I don't plan to be the decision-maker for this project,
as I'm more interested in building consensus among the co-authors. I
can certainly poll that consensus and update the KIP to keep the
project moving, and any other co-author can do the same. And to set an
example, I'll clarify your questions and for anything that I agree
with, I'll ask that you make the update to the KIP, so that the KIP
captures your understanding of the problem and your requirements. If
you don't get the chance to make the changes yourself, I'll make sure
they get included eventually, as they're very good ideas :)

For your remaining questions:

M1: I was trying to draw analogies to databases, but your suggested
properties are much more compelling and informative. I'd love it if
you added some formalism here, so that we have a better grasp on what
we're trying to accomplish. +1
M2: I think the "asynchronous" problem corresponds to the goal of
"exactly once semantics" but the two are not obviously opposites. I
think the MM2 deficiencies could focus less on the architecture
(asynchronicity) and more on the user-facing effect (semantics). +1
M3: I had a "non-goals" section that ended up becoming the "rejected
alternatives" section instead. If you have some non-goals in mind,
please add them.
M4+M5: I think it's too early to nail down the assumptions directly,
but if you believe that "separate operators of source and target" is a
requirement, that would be good to write down. +1
M6: That is a concerning edge case, and I don't know how to handle it.
I was imagining that there would be a many:many relationship of
clusters and links, but I understand that the book-keeping of that
decision may be significant.
M7: I think this may be appropriate to cover in a "user story" or
"example usages". I naturally thought that the feature would describe
some minimal way of linking two topics, and the applications
(combining multiple links, performing failovers, or running
active-active, etc) would be left to users to define. I included the
regex configurations because I imagine that creating 100s or 1000s of
links would be unnecessarily tedious. The feature may also encode
those use-cases directly as first-class citizens as well.

U1: These are states that can happen in reality, and I meant for that
section to imply that we should expect these states and model them for
operations and observability.

D1: I think I may have introduced this confusion by trying to be
terse. I imagined that there will be two different topics on the
source and target, which would be synced to have the same
configuration contents, similar to MM2's implementation. This would
allow for the replication link to be permanently disconnected and the
target topic to become just a regular topic, Later, a new replication
link and new target topic (with another separate topic-id) can be
created to rebuild the replication. I also thought that it was
possible that two clusters had already chosen the same topic-id, and
that attempting to interpret one topic-id in two different clusters
was error-prone. As far as replicating __cluster_metadata: I hadn't
considered that, but that might be required depending on the semantics
we choose.
D2: Thanks, that's a good clarification. Uptime and bandwidth should
be assumed to be lower, and latency should be assumed to be higher. +1
D3: I included this restriction because it would not be transparent to
source consumers. They would need special support for connecting to
brokers from multiple clusters, with potentially distinct metadata.

Thanks so much!
Greg

On Mon, Oct 2, 2023 at 4:24 PM Tom Bentley  wrote:
>
> Hi Greg,
>
> Thanks for this KIP! It is obviously very ambitious, but it's great to have
> a conversation about it.
>
> I'll start with some general points:
>
> Do you have a plan in mind for how to proceed with elaborating this KIP?
> While I like how you're involving the community in elaborating the KIP, I
> think there is a danger, which is more likely with this inclusive approach,
> in trying to attempt too much at once.
>
> In my opinion someone needs to take the difficult decisions necessary to
> limit the initial scope (and, just as importantly, communicate that
> clearly) in order to maximise the chances of actually getting something
> accepted and implemented. Can we assume that you're that person? Defining
> the what and how of the metadata replication, and the log replication seem
> to me to be the core of what you're trying to achieve here. We should make
> anything that is not crucial to that (i.e. NAT punching) a non-goal of this
> KIP. Future KIPs can easily add those features.
>
> I also had a 

Re: [DISCUSS] KIP-986: Cross-Cluster Replication

2023-10-02 Thread Tom Bentley
Hi Greg,

Thanks for this KIP! It is obviously very ambitious, but it's great to have
a conversation about it.

I'll start with some general points:

Do you have a plan in mind for how to proceed with elaborating this KIP?
While I like how you're involving the community in elaborating the KIP, I
think there is a danger, which is more likely with this inclusive approach,
in trying to attempt too much at once.

In my opinion someone needs to take the difficult decisions necessary to
limit the initial scope (and, just as importantly, communicate that
clearly) in order to maximise the chances of actually getting something
accepted and implemented. Can we assume that you're that person? Defining
the what and how of the metadata replication, and the log replication seem
to me to be the core of what you're trying to achieve here. We should make
anything that is not crucial to that (i.e. NAT punching) a non-goal of this
KIP. Future KIPs can easily add those features.

I also had a few specific points:

Motivation
M1. I don't find the "logical replication" vs "physical replication"
particularly helpful. I think one key property is "offset preserving",
which is also self-explanatory. Slightly more generally, we could define
the concept of "consumer transparency", i.e. a consumer could reconnect to
either cluster and observe the same sequences of records (same order, same
offsets, and same transaction visibility). Consumer transparency requires
synchronous replication, but offset preserving does not.
M2. In the motivation you mention that MM offers asynchronous replication,
but the Goals subsection doesn't mention support for synchronous
replication. We should be clear which (or both) we're aiming for.
M3. A Non-Goals section would be useful, especially for a KIP that's large
and ambitious like this one.
M4. It might also be worth having a list of Assumptions. Here we could list
all the things we want to assume in order to make the initial KIP feasible.
M5. For example we should be explicit about whether or not it is assumed
that the same people are operating (and thus have visibility into) both
clusters.
M6. One thing worth calling out is whether the clusters themselves are in a
leader/follower relationship (e.g. the DR scenario), or whether this is a
topic-level concern. I guess it's topic level from the topic and consumer
group regexes. But this has consequences we should explore. For example
what if a transaction includes records in topics X and Y, where X is
replicated but Y is not?
M7. I think you should be clear about whether this leader/follower
relationship can be reversed, and in what circumstances. In the user
interface section you talk about "disconnected", but not this kind of
fail-back.


User interface
U1. "Links can be temporarily or permanently disconnected." Are you
describing a fact about the network between the two clusters, or is this
disconnection something actively managed by the system, or by the operator?

Data semantics
D1. The KIP says "both cross-cluster topics and intra-cluster replicas:
Have the same configuration as their source" but you also say
"cross-cluster replicas: Have a separate topic-id", this seems like a
contradiction, on the face of it. It seems like there's a whole host of
devils in the detail behind this. It implies replication of (some of) the
__cluster_metadata, I think, but not all (since you say ACLs are not
replicated). If that's right, then what does it imply about referential
integrity between metadata records? i.e. what if metadata record A (which
is replicated) references record B (which is not)? Even if this is not
possible by design initially, how does it constrain the future evolution of
metadata record schemas? Is any such metadata replication going to be
transaction preserving? If the topic ids can differ then what is
responsible for the mapping and rewriting of metadata records which include
topic ids?
D2. "The network path between Kafka clusters is assumed to be less reliable
than the intra-cluster network," we should be explicit about whether or not
we're assuming similar network latencies and bandwidth for the
inter-cluster network links as for the in-cluster ones.
D3 "Are not eligible for fetch-from-follower on the source cluster" the
reason for this isn't immediately apparent to me.

Thanks again,

Tom

On Tue, 3 Oct 2023 at 09:37, Greg Harris 
wrote:

> Hi all,
>
> I've opened an extremely early draft of the Cross-Cluster Replication
> feature, and I would like to invite any and all co-authors to expand
> on it. Find the page here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-986%3A+Cross-Cluster+Replication
>
> This is not strictly an invitation to "review" the KIP, as the
> document has much less detail than other KIPs of similar complexity.
> But if you are knowledgeable in this area, some early sanity checks
> would be greatly appreciated.
>
> I've included a "shopping list" of properties that appear to me to be
> desirable, but I