> transactional messages [...] could result in frequent writes to the offset mapping topic.
Becket, I think we could limit writes to a max frequency to ameliorate this problem. > I am wondering if users can just seek by timestamp and get a more precise mapping [...] > I assume that MM2 will mirror the timestamps from source to target without being changed Yes, MM2 passes timestamps along, and also any headers. The timestamps are useful for measuring replication lag etc, but they are not particularly useful for consumer migration (failover etc). I can expound on this further if you like, but in practice offsets are better if you have them. > I don't think this should block MM2 given there are a lot of other benefits already Thanks! I appreciate your support. Ryanne On Fri, Dec 28, 2018 at 9:54 PM Becket Qin <becket....@gmail.com> wrote: > Hi Ryanne, > > Thanks for the reply. You are right. The topic naming and ACL sanity check > should probably be a separate discussion. > > Regarding the offset mapping. Thought about this a bit more. We may need to > consider the cases such as logs including transactional messages. In that > case, the offset mapping may not be contiguous due to the existence of the > transaction control messages. It could result in frequent writes to the > offset mapping topic. > > I don't think this should block MM2 given there are a lot of other benefits > already. That said, if we use periodic offset mapping records, I am > wondering if users can just seek by timestamp and get a more precise > mapping (at millisecond granularity) from the source cluster to the target > cluster. But admittedly, this approach has its own limitations such as > users are expected to use LogAppendTime for the original topic. (BTW, I > assume that MM2 will mirror the timestamps from source to target without > being changed) > > Thanks, > > Jiangjie (Becket) Qin > > On Thu, Dec 27, 2018 at 1:16 AM Ryanne Dolan <ryannedo...@gmail.com> > wrote: > > > Becket, this is great feedback, thanks. > > > > > having a reserved character for the topics is probably something worth > > doing in general > > > > Agreed, but we shouldn't make this a requirement for MM2, or else it > > wouldn't work with older versions of Kafka, complicating adoption, > testing > > etc. In particular, we'll want to prove MM2 against production workloads > > without first upgrading Kafka brokers. > > > > I think we should 1) make the separator configurable in MM2, defaulting > to > > a period for now, 2) in a separate KIP, propose a special separator > > character as you suggest, 3) maybe update the default at some point. > > > > We can endeavor to do this all in the same release, which would have the > > effect you want. > > > > > It might be better to add a config like allowAclMismatch to let user > > decide what should be the right behavior, i.e. either fail a mirror if > ACL > > mismatch, or mirror it with different ACLs. > > > > What would it mean to "fail a mirror"? I think it would be strange if > > replication suddenly stopped after someone changes an ACL somewhere. > > > > I think for users that want ACLs to mismatch, they'd just disable > > sync.topic.acls and manage ACLs themselves. I want MM2 to do the right > > thing by default, but I don't think it should be responsible for > enforcing > > policies or protecting against changes beyond its control and purview. > > > > > seems possible to achieve per message granularity, given that there is > a > > single writer to the remote topic [...] if the returned target offset is > > different from the expectation [...], MM2 emit a new mapping message to > the > > offset mapping topic > > > > This is a fantastic idea. With this approach, we can reduce the recovery > > period to a very small number of offsets -- effectively the same as the > > latest commit. This isn't exactly one-to-one offset translation, but as I > > said we don't really need that -- we just want to get as close to the > > latest consumer commit as possible without going past it, which your > > approach does very well. Thanks for this improvement. > > > > Ryanne > > > > > > On Tue, Dec 25, 2018 at 1:38 AM Becket Qin <becket....@gmail.com> wrote: > > > > > Hi Ryanne, > > > > > > Thanks for the reply. Please see a few more comments below. > > > > > > 1. Reserved character for topic names. > > > > > > > I think the limitations on topic names are imposed by Kafka itself > > > (beyond > > > > those from Zookeeper and the file system), so it might be possible to > > > add a > > > > reserved character for this purpose. This would essentially promote > the > > > > concept of "remote topics" from MirrorMaker to Kafka in general, > which > > > > might be nice. In particular, we could add native support for remote > > > topics > > > > to KafkaConsumer if we had a baked-in way to distinguish remote vs > > > > non-remote topics. Definitely worth investigating for a future KIP :) > > > > > > I do not worry much about the consumer side because it is fairly simple > > to > > > expand the subscription from a classic topic to a local + remote topic, > > as > > > long as users know the rule. I am more concerned about having a way to > > > ensure the users can safely use the remote topic without potential > > issues. > > > There were some earlier discussion about the hierarchical topic names / > > > topic namespaces. Those are probably too much in our context, but > having > > a > > > reserved character for the topics is probably something worth doing in > > > general. It seems simple enough and will help address the potential > > > confusion between local / remote topic names. > > > > > > 2. ACLs > > > I think in some cases user may want to have different ACL in local and > > > remote topics, but in some other cases, users may want to make sure > they > > > are the same to avoid unexpected behaviors. It might be better to add a > > > config like allowAclMismatch to let user decide what should be the > right > > > behavior, i.e. either fail a mirror if ACL mismatch, or mirror it with > > > different ACLs. > > > > > > 3. Offset mapping between source and destination Kafka cluster. > > > I haven't thought about this thoroughly, but seems possible to achieve > > per > > > message granularity, given that there is a single writer to the remote > > > topic. What we can do the is the following: > > > 1. For the first message MM2 mirrors, it will always emit a [source > > offset, > > > target offset] mapping to the offset mapping topic. (.e.g. (99 -> 199) > > > 2. After that, MM2 expect the offset in the destination partition to > > > increment one by one, corresponding to each message mirrored from > source. > > > (100 -> 200, 101 -> 201, etc...) > > > 3. At any given point, if the returned target offset is different from > > the > > > expectation (very likely larger, otherwise there is message loss), MM2 > > emit > > > a new mapping message to the offset mapping topic. ( supposedly, if 99 > -> > > > 199, then MM2 expect 199 -> 299, but if 199 -> 300, MM2 emits the pair > > (199 > > > -> 300)) > > > > > > In this case, for any committed source offset, the target offset can be > > > determined by doing the following: > > > 1. Find the offset mapping entry which contains the source offset that > is > > > closest but no larger than the committed source offset. (e.g. committed > > > offsets 150 will be mapped to the entry (99 -> 199)) > > > 2. Add a offsets difference because we know since that entry the > offsets > > > are increasing one by one. (target offsets = 199 + (150 -99) = 250) > > > > > > If the target offset is greater than the log end offset of the > partition, > > > that means that message has not been mirrored yet. > > > > > > Thanks, > > > > > > Jiangjie (Becket) Qin > > > > > > On Thu, Dec 20, 2018 at 6:00 AM Ryanne Dolan <ryannedo...@gmail.com> > > > wrote: > > > > > > > Becket, thanks for taking a look. > > > > > > > > > 1. Only relying on the topic naming seems a little fragile. > > > > > > > > I think the limitations on topic names are imposed by Kafka itself > > > (beyond > > > > those from Zookeeper and the file system), so it might be possible to > > > add a > > > > reserved character for this purpose. This would essentially promote > the > > > > concept of "remote topics" from MirrorMaker to Kafka in general, > which > > > > might be nice. In particular, we could add native support for remote > > > topics > > > > to KafkaConsumer if we had a baked-in way to distinguish remote vs > > > > non-remote topics. Definitely worth investigating for a future KIP :) > > > > > > > > > 2. For ACLs, is it possible that the local topic and remote topic > > have > > > > different ACLs? Will the connector do the sanity check? > > > > > > > > MM2 won't do full two-way sync of ACLs between clusters, but instead > > will > > > > simply propagate ACL changes from source topics to their downstream > > > remote > > > > topics. For example, if a principal can READ from topic1 in cluster > A, > > > the > > > > same principal should be able to READ A.topic1 in cluster B. > Otherwise, > > > > migrating a consumer from A -> B wouldn't work reliably. > > > > > > > > OTOH, say a super-user force-changes the ACL for a remote topic, that > > > > change would not be sync'd upstream to the source topic. In all > cases, > > > the > > > > "normal" source topics are the source-of-truth for ACL and > > configuration > > > of > > > > remote topics managed by MM2. > > > > > > > > So to your specific question: no, MM2 will not sanity-check remote > > > topics, > > > > since MM2 should be the only principal making any changes to remote > > > topics, > > > > just as MM2 is the only principal allowed to WRITE to remote topics. > > > > > > > > > 3. For the checkpoint. Because the consumer may commit at arbitrary > > > > offset. Does the connector need to keep a mapping between each source > > > > offset to destination offset? If so how would that be done? > > > > > > > > Offset translation will not be possible for arbitrary offsets. > Instead, > > > > checkpoints simply provide recent offsets that are safe for a foreign > > > > consumer to seek to. To accomplish this, MirrorSourceConnector will > > > > periodically (every checkpoint.interval.seconds) emit local-remote > > offset > > > > pairs for the records being replicated. The connector knows this > > mapping > > > > because it gets upstream offsets from the consumer and corresponding > > > > downstream offsets from the producer ACK. Then > > MirrorCheckpointConnector > > > > can use this sparse mapping when emitting checkpoints to the target > > > > cluster. > > > > > > > > For example, if a consumer group has committed up to offset 100, and > > the > > > > connector knows that offset 99 is equivalent to offset 199 on the > > target > > > > cluster, then the connector can safely emit a checkpoint for that > > > consumer > > > > group with target offset 199 == source offset 99, since 99 < 100. > > Notice > > > > the consumer is actually past 100 on the source cluster, but it's > only > > > safe > > > > to seek to 199 on the target cluster. > > > > > > > > If we checkpoint every 5 seconds in this way, we can be sure that a > > > > failed-over consumer won't end up re-processing much more than 5 > > seconds > > > of > > > > data. > > > > > > > > Thanks for the great questions. > > > > Ryanne > > > > > > > > > > > > On Thu, Dec 13, 2018 at 12:07 AM Becket Qin <becket....@gmail.com> > > > wrote: > > > > > > > > > Hi Ryanne, > > > > > > > > > > Thanks for the KIP. In general, I think the proposal makes a lot of > > > sense > > > > > to me. Unifying Kafka clusters in different locations has been a > very > > > > > important scenario. Right now open source community does not have a > > > > > standard solution to that. > > > > > > > > > > A few comments/questions are following: > > > > > > > > > > 1. Only relying on the topic naming seems a little fragile. It > seems > > > > > difficult to prevent conflict names. One workaround might be > adding a > > > > > config to the broker, preventing topics with character such as "_" > or > > > "." > > > > > from being created, unless it is from some specific client such as > > MM2 > > > > > admin clients. > > > > > > > > > > 2. For ACLs, is it possible that the local topic and remote topic > > have > > > > > different ACLs? Will the connector do the sanity check? > > > > > > > > > > 3. For the checkpoint. Because the consumer may commit at arbitrary > > > > offset. > > > > > Does the connector need to keep a mapping between each source > offset > > to > > > > > destination offset? If so how would that be done? > > > > > > > > > > Thanks, > > > > > > > > > > Jiangjie (Becket) Qin > > > > > > > > > > On Thu, Dec 13, 2018 at 8:23 AM Ryanne Dolan < > ryannedo...@gmail.com> > > > > > wrote: > > > > > > > > > > > > Consuming system restarts and restates from compacted topics, > > using > > > > > > *.account_state > > > > > > > > > > > > Michael, I think I understand your concern. With MM2's remote > topic > > > > > > concept, it's the consumer's responsibility to aggregate records > > > across > > > > > > .*account_state. MM2 will not merge these for you (by design). > This > > > > > > complicates your consumer that expects a single topic to be > > compacted > > > > > > across all clusters, since log-compaction does not span topics. > > > > > > > > > > > > Notice that MM2 isn't really breaking anything here. The problem > is > > > > that > > > > > > you are using MirrorMaker to aggregate records from multiple > > clusters > > > > > into > > > > > > a single topic, which MM2 will not do by default. > > > > > > > > > > > > It's easy to accomplish the same thing with MM2 though. You have > a > > > > couple > > > > > > options: > > > > > > > > > > > > 1) turn off topic renaming or use "legacy mode". MM2 won't get in > > > your > > > > > way, > > > > > > but it won't help much for DR, failover/failback etc. You're on > > your > > > > own > > > > > :) > > > > > > > > > > > > 2) Aggregate your topic downstream of MM2. Turn .*account_state > > into > > > > > > account_state.aggregate, which itself is log-compacted. KTables > > make > > > > this > > > > > > trivial! > > > > > > > > > > > > With (2) you still get the nice DR semantics. The KTable will > > > represent > > > > > the > > > > > > latest account states aggregated across all clusters. If your > > > producers > > > > > > need to migrate/failover/failback, the KTable in each region will > > be > > > > > > eventually consistent, and downstream consumers won't notice. I'd > > > > > recommend > > > > > > blacklisting .*aggregate in MM2 so these don't get replicated > > > > themselves. > > > > > > > > > > > > TBH, I thought about including this as a feature in MM2, s.t. you > > > would > > > > > get > > > > > > a foo.aggregate topic automatically. Personally, I think Streams > is > > > > > better > > > > > > suited than Connect for this, but I won't rule it out for a > future > > > KIP. > > > > > > > > > > > > Thanks again! > > > > > > Ryanne > > > > > > > > > > > > > > > > > > > > > > > > On Wed, Dec 12, 2018 at 2:49 PM Michael Pearce < > > > michael.pea...@ig.com> > > > > > > wrote: > > > > > > > > > > > > > Ok so you have your transaction system running master master. > > > > > > > > > > > > > > You have three dcs, regions where-ever. > > > > > > > > > > > > > > Accounts are distributed across area based on an number of > > factors > > > > > > > > > > > > > > Account A + B are in Ireland > > > > > > > Account C + D are in Germany > > > > > > > Account E are in UK > > > > > > > > > > > > > > Lets prefix call kafka cluster in Ireland Z, Germany Y and UK X > > > > > > > > > > > > > > So now I have topics: > > > > > > > > > > > > > > Z.account_state (holding state for accounts A + B) > > > > > > > Y.account_state (holding state for accounts C + D) > > > > > > > X.account_state (holding state for account E) > > > > > > > > > > > > > > To get a full view of all accounts a consumer would consume > > > > > > *.account_state > > > > > > > Great! > > > > > > > > > > > > > > Now image we take a planned or unplanned outage in the UK. > > > > > > > We move/shift the transactional processing of account E to > > Ireland. > > > > > > > > > > > > > > Now we end up with > > > > > > > Z.account_state (holding state for accounts A + B + current > > state > > > > for > > > > > > > account E) > > > > > > > Y.account_state (holding state for accounts C + D) > > > > > > > X.account_state (holding outdated state for account E) > > > > > > > > > > > > > > > > > > > > > Consuming system restarts and restates from compacted topics, > > using > > > > > > > *.account_state > > > > > > > > > > > > > > It will get both current state and also outdated state for E, > > which > > > > > state > > > > > > > would it use? And this gets worse as it scales and you move the > > > > > > processing > > > > > > > of accounts around over time. > > > > > > > > > > > > > > > > > > > > > Likewise the issue is the same without compacted state. > > > > > > > > > > > > > > Imagine order of state change is important by account globally, > > how > > > > do > > > > > > you > > > > > > > re-order the state changes? As order guarantee can be given > > within > > > a > > > > > > single > > > > > > > topic partition only. You only care for order by the account > (not > > > for > > > > > > > offets being same, we simply care that updates are in order and > > > > latest > > > > > > > state is at head on restart we ignore offsets). So it doesn’t > > > matter > > > > if > > > > > > in > > > > > > > Z cluster you see update A1, A2, B1, A3, B2, E1, E2 and in > > cluster > > > Y > > > > > you > > > > > > > see B1,B2, A1,E1, A2, A3, E2 as the ordering by of the updates > > > > account > > > > > is > > > > > > > preserved. > > > > > > > > > > > > > > With the topic solution your suggesting we would have no way > true > > > way > > > > > of > > > > > > > replaying and re-constituting the order between X.account_state > > and > > > > > > > Z.account_state topics in the case of E, as messages will be in > > > > > different > > > > > > > topics and partitions. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -----Original Message----- > > > > > > > From: Ryanne Dolan <ryannedo...@gmail.com> > > > > > > > Sent: Wednesday, December 12, 2018 4:37 PM > > > > > > > To: dev@kafka.apache.org > > > > > > > Subject: Re: [DISCUSS] KIP-382: MirrorMaker 2.0 > > > > > > > > > > > > > > > You haven’t described how one would handle the ordering > issues > > > and > > > > > also > > > > > > > the compaction issues where transactional processing is > > > master-master > > > > > in > > > > > > > regions, where the processing is sticky to region but of > failure > > or > > > > > > > planned, processing of certain accounts move regions. > > > > > > > > > > > > > > Michael, a couple points: > > > > > > > > > > > > > > - With "remote topics", the order of events is consistent > between > > > > > > clusters. > > > > > > > Cluster A's "topic1" is the same records in the same order as > > > cluster > > > > > B's > > > > > > > "A.topic1". The plan is to enable exactly-once semantics within > > MM2 > > > > so > > > > > > > there aren't additional dupes either (though I believe this > will > > > > > require > > > > > > > support within Connect and a separate KIP). > > > > > > > > > > > > > > - A consumer that is only interested in events produced in a > > > > particular > > > > > > > region can migrate to a cluster in a different region by > updating > > > > it's > > > > > > > subscription accordingly. For example, a consumer in us-west > > > > processing > > > > > > > events local to us-west would consume topics like "topic1" (a > > > normal > > > > > > > topic). If you migrate this consumer to us-east, it would need > to > > > > > > subscribe > > > > > > > to "us-west.topic1" instead. It's clear from the naming > > convention > > > > that > > > > > > > "us-west.topic1" is a replicated topic with records originating > > > from > > > > a > > > > > > > remote cluster. > > > > > > > > > > > > > > I'm not sure I understand your concern w.r.t compacted topics > and > > > > > state. > > > > > > > Can you elaborate? > > > > > > > > > > > > > > Ryanne > > > > > > > > > > > > > > > > > > > > > On Wed, Dec 12, 2018 at 8:41 AM Michael Pearce < > > > > michael.pea...@ig.com> > > > > > > > wrote: > > > > > > > > > > > > > > > Ryanne, > > > > > > > > > > > > > > > > You haven’t described how one would handle the ordering > issues > > > and > > > > > also > > > > > > > > the compaction issues where transactional processing is > > > > master-master > > > > > > in > > > > > > > > regions, where the processing is sticky to region but of > > failure > > > or > > > > > > > > planned, processing of certain accounts move regions. > > > > > > > > > > > > > > > > Also I ask that you keep compatibility of the handler api > > > interface > > > > > in > > > > > > MM > > > > > > > > into MM2. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -----Original Message----- > > > > > > > > From: Ryanne Dolan <ryannedo...@gmail.com> > > > > > > > > Sent: Wednesday, December 12, 2018 6:41 AM > > > > > > > > To: dev@kafka.apache.org > > > > > > > > Subject: Re: [DISCUSS] KIP-382: MirrorMaker 2.0 > > > > > > > > > > > > > > > > > One based on hops using headers, and another based on topic > > > > naming. > > > > > > > > > > > > > > > > Michael, this was also suggested by Alex Mironov. Other > > > replication > > > > > > > engines > > > > > > > > use headers as you describe, but there are several issues > with > > > this > > > > > > > > approach: > > > > > > > > > > > > > > > > - The Connect framework provides Transformations that could > be > > > used > > > > > for > > > > > > > > this purpose, so MM2 doesn't necessarily need to address this > > > > feature > > > > > > > > directly. For example, a Transformation could tag each record > > > with > > > > > its > > > > > > > > source cluster alias or could decrement a hop/TTL value if > you > > > > like. > > > > > > > > > > > > > > > > - We want MM2 to benefit from "shallow iteration", meaning > that > > > > e.g. > > > > > > > > compressed message sets should pass through MM2 without being > > > > > > > decompressed > > > > > > > > and decomposed. In many cases, this can result in significant > > > > > > performance > > > > > > > > gains. But since these message sets are not decomposed into > > > > > individual > > > > > > > > messages and so on, we cannot reliably tag each message that > > > passes > > > > > > > > through. > > > > > > > > > > > > > > > > - Fundamentally, I believe it is a bad idea to treat topics > on > > > > > > different > > > > > > > > clusters as if they are the same topic. If two clusters have > > > topics > > > > > of > > > > > > > the > > > > > > > > same name, they are still inherently different topics, as > they > > > > exist > > > > > on > > > > > > > > different clusters, with potentially different records and > > > > certainly > > > > > > > > different offsets. Moreover, there is no robust way to keep > > such > > > > > topics > > > > > > > > consistent and in sync. > > > > > > > > > > > > > > > > - Some applications are interested in events from a > particular > > > data > > > > > > > > center/region/cluster, while other applications will want to > > > > process > > > > > > > events > > > > > > > > regardless of where they are produced. Indeed, a common > > use-case > > > > for > > > > > > > > building out multiple clusters in the first place is to > support > > > > this > > > > > > sort > > > > > > > > of geolocation-aware processing and aggregation. It sounds > like > > > > your > > > > > > org > > > > > > > > attempts to make topics the same everywhere, which is > > undesirable > > > > in > > > > > > many > > > > > > > > cases. > > > > > > > > > > > > > > > > - Advanced operations such as automatic failover and failback > > > rely > > > > on > > > > > > the > > > > > > > > ability to segregate records based on their cluster of > origin, > > > > while > > > > > > > > preserving order within each topic-partition. This is > extremely > > > > > > difficult > > > > > > > > if your app's producers and MM's producers "cross streams" by > > > > writing > > > > > > > into > > > > > > > > the same topic. (Your mention of "ensure inflight processing > at > > > the > > > > > old > > > > > > > > region ceased" etc is hand-holding symptomatic of this > > problem.) > > > > > > > > > > > > > > > > - With a consistent naming convention in place (along with > > > > > > checkpointing, > > > > > > > > heartbeats and the other "best practices" mentioned in the > > KIP), > > > we > > > > > can > > > > > > > > build tooling that understands multi-cluster environments. > For > > > > > example, > > > > > > > the > > > > > > > > KIP describes a utility for translating consumer offsets > > between > > > > > > > clusters. > > > > > > > > This will enable operators to migrate consumer groups between > > > > > clusters > > > > > > > > without knowing anything about the topics involved. > > > > > > > > > > > > > > > > That all said, I am sure some organizations will want to > apply > > > > their > > > > > > own > > > > > > > > conventions, and I don't believe MM2 should get in the way of > > > that. > > > > > > > > > > > > > > > > Thanks again! > > > > > > > > Ryanne > > > > > > > > > > > > > > > > On Tue, Dec 11, 2018 at 7:20 PM michael.andre.pearce > > > > > > > > <michael.andre.pea...@me.com.invalid> wrote: > > > > > > > > > > > > > > > > > Another benefit of using hops vs topic naming (also how we > > > > > currently > > > > > > do > > > > > > > > > master master in my org) > > > > > > > > > You have a transactional processing app that's multi > > regioned. > > > So > > > > > for > > > > > > > > sake > > > > > > > > > of discussion all A and B accounts process normally in > > ireland > > > > > region > > > > > > > > all C > > > > > > > > > and D in germany region and all E in Uk region. Increase > > > regions > > > > to > > > > > > > > across > > > > > > > > > the globe almost one per financial centre where close > regions > > > > might > > > > > > > mesh > > > > > > > > > together one to one and continental regions form some > > backbones > > > > > > trunks. > > > > > > > > > If uk region goes down planned or unplanned we move the > > > > > transactional > > > > > > > > > routing and processing to germany region. During this flip > > over > > > > we > > > > > > > ensure > > > > > > > > > inflight processing at the old region ceased and restate > > > account > > > > > > states > > > > > > > > > from topics before further processing thus ensuring no out > of > > > > order > > > > > > > > message > > > > > > > > > production by account. > > > > > > > > > The with using region broker or some other topic prefixes > > this > > > > will > > > > > > > mean > > > > > > > > > now i have a topic for uk region with some E account data > and > > > > after > > > > > > > > process > > > > > > > > > movement will end up with E accounts in the topic for > germany > > > > > region. > > > > > > > Now > > > > > > > > > ordering is lost. Worst still if compacted topic i will > have > > > two > > > > > > states > > > > > > > > as > > > > > > > > > state would be in two topics. > > > > > > > > > With using hops and single named topic (as we do already in > > our > > > > own > > > > > > > > custom > > > > > > > > > handlers which do hop logic with todays mirrormakers) we > can > > > > avoid > > > > > > this > > > > > > > > > issue entirely as ordering by account is preserved as is > all > > in > > > > one > > > > > > > topic > > > > > > > > > still and also when using compacted we have only one state. > > > > > > > > > > > > > > > > > > Before you say why not name topics a. b. I use that to > > simplify > > > > the > > > > > > > case > > > > > > > > > to be able describe it. Accounts are autogenrated e.g. > > A16E45T1 > > > > and > > > > > > > > C43F4SA > > > > > > > > > could be processing in germany region currently and C43F2SA > > > could > > > > > be > > > > > > in > > > > > > > > uk > > > > > > > > > region currently. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Sent from my Samsung Galaxy smartphone. > > > > > > > > > -------- Original message --------From: Andrew Otto < > > > > > > > o...@wikimedia.org> > > > > > > > > > Date: 11/12/2018 14:28 (GMT+00:00) To: > > dev@kafka.apache.org > > > > > > Subject: > > > > > > > > > Re: [DISCUSS] KIP-382: MirrorMaker 2.0 > > > > > > > > > Wikimedia currently implements 'master <-> master' > > replication > > > by > > > > > > > > manually > > > > > > > > > prefixing topics with datacenter names, and then > configuring > > > > > > > MirrorMaker > > > > > > > > to > > > > > > > > > only replicate topics that begin with a DC name to another. > > > > > > > > > > > > > > > > > > While having topics named with topological details is > > > > manageable, I > > > > > > > > > wouldn't say it is desirable. It pushes knowledge of the > > > > > replication > > > > > > > > > topology up to clients. Even if MirrorMaker was the one > > doing > > > > the > > > > > > > topic > > > > > > > > > prefixing, downstream consumers of a group of replicated > > topics > > > > are > > > > > > > still > > > > > > > > > going to have to know to subscribe to the correctly > prefixed > > > > > topics. > > > > > > > > > > > > > > > > > > If possible I'd much prefer header + hops based replication > > > > rather > > > > > > than > > > > > > > > > lots of renamed topics. But either way, this KIP would be > > > > > > tremendously > > > > > > > > > useful to us so I support it all the way! :) > > > > > > > > > > > > > > > > > > On Tue, Dec 11, 2018 at 5:32 AM Michael Pearce < > > > > > > michael.pea...@ig.com> > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > So this is indeed what using headers with hops avoids is > > > > creating > > > > > > > lots > > > > > > > > > and > > > > > > > > > > lots of topics __, so you can have more complex topology > > > > setups. > > > > > > > > > > > > > > > > > > > > I ask why not support having two ways of setting up and > > > closing > > > > > the > > > > > > > > door? > > > > > > > > > > > > > > > > > > > > One based on hops using headers, and another based on > topic > > > > > naming. > > > > > > > > After > > > > > > > > > > all flexibility is what we want its for end users how to > > use > > > > > right? > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On 12/7/18, 8:19 PM, "Ryanne Dolan" < > > ryannedo...@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > Michael, thanks for the comments! > > > > > > > > > > > > > > > > > > > > > would like to see support for this to be done by > > hops, > > > > as > > > > > > well > > > > > > > > > [...] > > > > > > > > > > This then allows ring (hops = number of brokers in > the > > > > ring), > > > > > > > mesh > > > > > > > > > > (every > > > > > > > > > > cluster interconnected so hop=1), or even a tree > (more > > > fine > > > > > > > grained > > > > > > > > > > setup) > > > > > > > > > > cluster topology. > > > > > > > > > > > > > > > > > > > > That's a good idea, though we can do this at the > topic > > > > level > > > > > > > > without > > > > > > > > > > tagging individual records. A max.hop of 1 would mean > > > > > > "A.topic1" > > > > > > > is > > > > > > > > > > allowed, but not "B.A.topic1". I think the default > > > behavior > > > > > > would > > > > > > > > > need > > > > > > > > > > to > > > > > > > > > > be max.hops = 1 to avoid unexpectedly creating a > bunch > > of > > > > > > > > D.C.B.A... > > > > > > > > > > topics > > > > > > > > > > when you create a fully-connected mesh topology. > > > > > > > > > > > > > > > > > > > > Looking ahead a bit, I can imagine an external tool > > > > computing > > > > > > the > > > > > > > > > > spanning > > > > > > > > > > tree of topics among a set of clusters based on > > > > inter-cluster > > > > > > > > > > replication > > > > > > > > > > lag, and setting up MM2 accordingly. But that's > > probably > > > > > > outside > > > > > > > > the > > > > > > > > > > scope > > > > > > > > > > of this KIP :) > > > > > > > > > > > > > > > > > > > > > ...standalone MirrorMaker connector... > > > > > > > > > > > ./bin/kafka-mirror-maker-2.sh --consumer > > > > > > > consumer.properties > > > > > > > > > > --producer producer.properties > > > > > > > > > > > > > > > > > > > > Eventually, I'd like MM2 to completely replace legacy > > MM, > > > > > > > including > > > > > > > > > the > > > > > > > > > > ./bin/kafka-mirror-maker.sh script. In the meantime, > > > it's a > > > > > > good > > > > > > > > idea > > > > > > > > > > to > > > > > > > > > > include a standalone driver. Something like > > > > > > > > > > ./bin/connect-mirror-maker-standalone.sh with the > same > > > > > > high-level > > > > > > > > > > configuration file. I'll do that, thanks. > > > > > > > > > > > > > > > > > > > > > I see no section on providing support for mirror > > maker > > > > > > > Handlers, > > > > > > > > > > today > > > > > > > > > > people can add handlers to have a little extra custom > > > logic > > > > > if > > > > > > > > > needed, > > > > > > > > > > and > > > > > > > > > > the handler api is public today so should be > supported > > > > going > > > > > > > > forwards > > > > > > > > > > so > > > > > > > > > > people are not on mass re-writing these. > > > > > > > > > > > > > > > > > > > > Great point. Connect offers single-message > > > transformations > > > > > and > > > > > > > > > > converters > > > > > > > > > > for this purpose, but I agree that we should honor > the > > > > > existing > > > > > > > API > > > > > > > > > if > > > > > > > > > > possible. This might be as easy as providing an > adapter > > > > class > > > > > > > > between > > > > > > > > > > connect's Transformation and mirror-maker's Handler. > > > Maybe > > > > > > file a > > > > > > > > > Jira > > > > > > > > > > ticket to track this? > > > > > > > > > > > > > > > > > > > > Really appreciate your feedback! > > > > > > > > > > > > > > > > > > > > Ryanne > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Thu, Dec 6, 2018 at 7:03 PM Michael Pearce < > > > > > > > > michael.pea...@ig.com > > > > > > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > Re hops to stop the cycle and to allow a range of > > multi > > > > > > cluster > > > > > > > > > > > topologies, see > > > > > > > > https://www.rabbitmq.com/federated-exchanges.html > > > > > > > > > > where > > > > > > > > > > > very similar was done in rabbit. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On 12/7/18, 12:47 AM, "Michael Pearce" < > > > > > > michael.pea...@ig.com> > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > Nice proposal. > > > > > > > > > > > > > > > > > > > > > > Some comments. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On the section around cycle detection. > > > > > > > > > > > > > > > > > > > > > > I would like to see support for this to be done > > by > > > > > hops, > > > > > > as > > > > > > > > > well > > > > > > > > > > e.g. > > > > > > > > > > > using approach is to use a header for the number of > > > hops, > > > > > as > > > > > > > the > > > > > > > > > mm2 > > > > > > > > > > > replicates it increases the hop count and you can > > make > > > > the > > > > > > mm2 > > > > > > > > > > configurable > > > > > > > > > > > to only produce messages onwards where hops are > less > > > than > > > > > x. > > > > > > > > > > > This then allows ring (hops = number of brokers > > in > > > > the > > > > > > > ring), > > > > > > > > > > mesh > > > > > > > > > > > (every cluster interconnected so hop=1), or even a > > tree > > > > > (more > > > > > > > > fine > > > > > > > > > > grained > > > > > > > > > > > setup) cluster topology. > > > > > > > > > > > FYI we do this currently with the current > mirror > > > > maker, > > > > > > > > using a > > > > > > > > > > custom > > > > > > > > > > > handler. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On the section around running a standalone > > > > MirrorMaker > > > > > > > > > connector > > > > > > > > > > > > > > > > > > > > > > I would suggest making this as easy to run as > the > > > > > > > > mirrormakers > > > > > > > > > > are > > > > > > > > > > > today, with a simple single sh script. > > > > > > > > > > > I assume this is what is proposed in section > > > "Running > > > > > > > > > > MirrorMaker in > > > > > > > > > > > legacy mode" but I would even do this before MM > would > > > be > > > > > > > removed, > > > > > > > > > > with a -2 > > > > > > > > > > > varient. > > > > > > > > > > > e.g. > > > > > > > > > > > ./bin/kafka-mirror-maker-2.sh --consumer > > > > > > > consumer.properties > > > > > > > > > > > --producer producer.properties > > > > > > > > > > > > > > > > > > > > > > Lastly > > > > > > > > > > > > > > > > > > > > > > I see no section on providing support for > mirror > > > > maker > > > > > > > > > Handlers, > > > > > > > > > > today > > > > > > > > > > > people can add handlers to have a little extra > custom > > > > logic > > > > > > if > > > > > > > > > > needed, and > > > > > > > > > > > the handler api is public today so should be > > supported > > > > > going > > > > > > > > > > forwards so > > > > > > > > > > > people are not on mass re-writing these. > > > > > > > > > > > > > > > > > > > > > > On 12/5/18, 5:36 PM, "Ryanne Dolan" < > > > > > > ryannedo...@gmail.com > > > > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > Sönke, > > > > > > > > > > > > > > > > > > > > > > > The only thing that I could come up with > is > > > the > > > > > > > > > limitation > > > > > > > > > > to a > > > > > > > > > > > single > > > > > > > > > > > offset commit interval > > > > > > > > > > > > > > > > > > > > > > Yes, and other internal properties, e.g. > > those > > > > used > > > > > > by > > > > > > > > the > > > > > > > > > > internal > > > > > > > > > > > consumers and producers, which, granted, > > > probably > > > > > are > > > > > > > not > > > > > > > > > > often > > > > > > > > > > > changed > > > > > > > > > > > from their defaults, but that apply to > > > Connectors > > > > > > > across > > > > > > > > > the > > > > > > > > > > > entire cluster. > > > > > > > > > > > > > > > > > > > > > > Ryanne > > > > > > > > > > > > > > > > > > > > > > On Wed, Dec 5, 2018 at 3:21 AM Sönke Liebau > > > > > > > > > > > <soenke.lie...@opencore.com.invalid> > wrote: > > > > > > > > > > > > > > > > > > > > > > > Hi Ryanne, > > > > > > > > > > > > > > > > > > > > > > > > when you say "Currently worker configs > > apply > > > > > across > > > > > > > the > > > > > > > > > > entire > > > > > > > > > > > cluster, > > > > > > > > > > > > which is limiting even for use-cases > > > involving > > > > a > > > > > > > single > > > > > > > > > > Kafka > > > > > > > > > > > cluster.", > > > > > > > > > > > > may I ask you to elaborate on those > > > > limitations a > > > > > > > > little? > > > > > > > > > > > > The only thing that I could come up with > is > > > the > > > > > > > > > limitation > > > > > > > > > > to a > > > > > > > > > > > single > > > > > > > > > > > > offset commit interval value for all > > running > > > > > > > > connectors. > > > > > > > > > > > > Maybe also the limitation to shared > config > > > > > > > providers.. > > > > > > > > > > > > > > > > > > > > > > > > But you sound like you had painful > > > experiences > > > > > with > > > > > > > > this > > > > > > > > > > before, > > > > > > > > > > > maybe > > > > > > > > > > > > you'd like to share the burden :) > > > > > > > > > > > > > > > > > > > > > > > > Best regards, > > > > > > > > > > > > Sönke > > > > > > > > > > > > > > > > > > > > > > > > On Wed, Dec 5, 2018 at 5:15 AM Ryanne > > Dolan < > > > > > > > > > > > ryannedo...@gmail.com> wrote: > > > > > > > > > > > > > > > > > > > > > > > > > Sönke, > > > > > > > > > > > > > > > > > > > > > > > > > > I think so long as we can keep the > > > > differences > > > > > > at a > > > > > > > > > very > > > > > > > > > > high > > > > > > > > > > > level (i.e. > > > > > > > > > > > > > the "control plane"), there is little > > > > downside > > > > > to > > > > > > > MM2 > > > > > > > > > and > > > > > > > > > > > Connect > > > > > > > > > > > > > coexisting. I do expect them to > converge > > to > > > > > some > > > > > > > > > extent, > > > > > > > > > > with > > > > > > > > > > > features > > > > > > > > > > > > from > > > > > > > > > > > > > MM2 being pulled into Connect whenever > > this > > > > is > > > > > > > > possible > > > > > > > > > > > without breaking > > > > > > > > > > > > > things. > > > > > > > > > > > > > > > > > > > > > > > > > > I could definitely see your idea re > > > > hierarchies > > > > > > or > > > > > > > > > > groups of > > > > > > > > > > > connectors > > > > > > > > > > > > > being useful outside MM2. Currently > > "worker > > > > > > > configs" > > > > > > > > > > apply > > > > > > > > > > > across the > > > > > > > > > > > > > entire cluster, which is limiting even > > for > > > > > > > use-cases > > > > > > > > > > involving > > > > > > > > > > > a single > > > > > > > > > > > > > Kafka cluster. If Connect supported > > > multiple > > > > > > > workers > > > > > > > > in > > > > > > > > > > the > > > > > > > > > > > same cluster, > > > > > > > > > > > > > it would start to look a lot like a MM2 > > > > > cluster. > > > > > > > > > > > > > > > > > > > > > > > > > > Ryanne > > > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Dec 4, 2018 at 3:26 PM Sönke > > Liebau > > > > > > > > > > > > > <soenke.lie...@opencore.com.invalid> > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > Hi Ryanne, > > > > > > > > > > > > > > > > > > > > > > > > > > > > thanks for your response! > > > > > > > > > > > > > > > > > > > > > > > > > > > > It seems like you have already done a > > lot > > > > of > > > > > > > > > > investigation > > > > > > > > > > > into the > > > > > > > > > > > > > > existing code and the solution design > > and > > > > all > > > > > > of > > > > > > > > what > > > > > > > > > > you > > > > > > > > > > > write makes > > > > > > > > > > > > > sense > > > > > > > > > > > > > > to me. Would it potentially be worth > > > adding > > > > > > this > > > > > > > to > > > > > > > > > > the KIP, > > > > > > > > > > > now that > > > > > > > > > > > > you > > > > > > > > > > > > > > had to write it up because of me > > anyway? > > > > > > > > > > > > > > > > > > > > > > > > > > > > However, I am afraid that I am still > > not > > > > > > entirely > > > > > > > > > > convinced > > > > > > > > > > > of the > > > > > > > > > > > > > > fundamental benefit this provides > over > > an > > > > > > > extended > > > > > > > > > > Connect > > > > > > > > > > > that has the > > > > > > > > > > > > > > following functionality: > > > > > > > > > > > > > > - allow for organizing connectors > into > > a > > > > > > > > hierarchical > > > > > > > > > > > structure - > > > > > > > > > > > > > > "clusters/us-west/..." > > > > > > > > > > > > > > - allow defining external Kafka > > clusters > > > to > > > > > be > > > > > > > used > > > > > > > > > by > > > > > > > > > > > Source and Sink > > > > > > > > > > > > > > connectors instead of the local > cluster > > > > > > > > > > > > > > > > > > > > > > > > > > > > Personally I think both of these > > features > > > > are > > > > > > > > useful > > > > > > > > > > > additions to > > > > > > > > > > > > > Connect, > > > > > > > > > > > > > > I'll address both separately below. > > > > > > > > > > > > > > > > > > > > > > > > > > > > Allowing to structure connectors in a > > > > > hierarchy > > > > > > > > > > > > > > Organizing running connectors will > grow > > > > more > > > > > > > > > important > > > > > > > > > > as > > > > > > > > > > > corporate > > > > > > > > > > > > > > customers adapt Connect and > > installations > > > > > grow > > > > > > in > > > > > > > > > size. > > > > > > > > > > > Additionally > > > > > > > > > > > > this > > > > > > > > > > > > > > could be useful for ACLs in case they > > are > > > > > ever > > > > > > > > added > > > > > > > > > to > > > > > > > > > > > Connect, as you > > > > > > > > > > > > > > could allow specific users access > only > > to > > > > > > > specific > > > > > > > > > > > namespaces (and > > > > > > > > > > > > until > > > > > > > > > > > > > > ACLs are added it would facilitate > > using > > > a > > > > > > > reverse > > > > > > > > > > proxy for > > > > > > > > > > > the same > > > > > > > > > > > > > > effect). > > > > > > > > > > > > > > > > > > > > > > > > > > > > Allow accessing multiple external > > > clusters > > > > > > > > > > > > > > The reasoning for this feature is > > pretty > > > > much > > > > > > the > > > > > > > > > same > > > > > > > > > > as > > > > > > > > > > > for a central > > > > > > > > > > > > > > Mirror Maker cluster, if a company > has > > > > > multiple > > > > > > > > > > clusters for > > > > > > > > > > > whatever > > > > > > > > > > > > > > reason but wants to have ingest > > > centralized > > > > > in > > > > > > > one > > > > > > > > > > system > > > > > > > > > > > aka one > > > > > > > > > > > > Connect > > > > > > > > > > > > > > cluster they would need the ability > to > > > read > > > > > > from > > > > > > > > and > > > > > > > > > > write > > > > > > > > > > > to an > > > > > > > > > > > > > arbitrary > > > > > > > > > > > > > > number of Kafka clusters. > > > > > > > > > > > > > > I haven't really looked at the code, > > just > > > > > poked > > > > > > > > > around > > > > > > > > > > a > > > > > > > > > > > couple of > > > > > > > > > > > > > minutes, > > > > > > > > > > > > > > but it appears like this could be > done > > > with > > > > > > > fairly > > > > > > > > > low > > > > > > > > > > > effort. My > > > > > > > > > > > > general > > > > > > > > > > > > > > idea would be to leave the existing > > > > > > configuration > > > > > > > > > > options > > > > > > > > > > > untouched - > > > > > > > > > > > > > > Connect will always need a "primary" > > > > cluster > > > > > > that > > > > > > > > is > > > > > > > > > > used > > > > > > > > > > > for storage > > > > > > > > > > > > of > > > > > > > > > > > > > > internal data (config, offsets, > status) > > > > there > > > > > > is > > > > > > > no > > > > > > > > > > need to > > > > > > > > > > > break > > > > > > > > > > > > > existing > > > > > > > > > > > > > > configs. But additionally allow > adding > > > > named > > > > > > > extra > > > > > > > > > > clusters > > > > > > > > > > > by > > > > > > > > > > > > specifying > > > > > > > > > > > > > > options like > > > > > > > > > > > > > > > > > > > external.sales_cluster.bootstrap_servers=... > > > > > > > > > > > > > > > > > > > > > external.sales_cluster.ssl.keystore.location=... > > > > > > > > > > > > > > > > > > > > > external.marketing_cluster.bootstrap_servers=... > > > > > > > > > > > > > > > > > > > > > > > > > > > > The code for status, offset and > config > > > > > storage > > > > > > is > > > > > > > > > > mostly > > > > > > > > > > > isolated in > > > > > > > > > > > > the > > > > > > > > > > > > > > > Kafka[Offset|Status|Config]BackingStore > > > > > classes > > > > > > > and > > > > > > > > > > could > > > > > > > > > > > remain pretty > > > > > > > > > > > > > > much unchanged. > > > > > > > > > > > > > > > > > > > > > > > > > > > > Producer and consumer creation for > > Tasks > > > is > > > > > > done > > > > > > > in > > > > > > > > > the > > > > > > > > > > > Worker as of > > > > > > > > > > > > > > KAFKA-7551 and is isolated in two > > > > functions. > > > > > We > > > > > > > > could > > > > > > > > > > add a > > > > > > > > > > > two more > > > > > > > > > > > > > > functions with an extra argument for > > the > > > > > > external > > > > > > > > > > cluster > > > > > > > > > > > name to be > > > > > > > > > > > > used > > > > > > > > > > > > > > and return fitting > consumers/producers. > > > > > > > > > > > > > > The source and sink config would then > > > > simply > > > > > > gain > > > > > > > > an > > > > > > > > > > > optional setting > > > > > > > > > > > > to > > > > > > > > > > > > > > specify the cluster name. > > > > > > > > > > > > > > > > > > > > > > > > > > > > I am very sure that I am missing a > few > > > > large > > > > > > > issues > > > > > > > > > > with > > > > > > > > > > > these ideas, > > > > > > > > > > > > I'm > > > > > > > > > > > > > > mostly back-of-the-napkin designing > > here, > > > > but > > > > > > it > > > > > > > > > might > > > > > > > > > > be > > > > > > > > > > > worth a > > > > > > > > > > > > second > > > > > > > > > > > > > > look. > > > > > > > > > > > > > > > > > > > > > > > > > > > > Once we decide to diverge into two > > > > clusters: > > > > > > > > > > MirrorMaker and > > > > > > > > > > > Connect, I > > > > > > > > > > > > > > think realistically the chance of > those > > > two > > > > > > ever > > > > > > > > > being > > > > > > > > > > > merged again > > > > > > > > > > > > > because > > > > > > > > > > > > > > they grow back together is > practically > > > > zero - > > > > > > > hence > > > > > > > > > my > > > > > > > > > > > hesitation. > > > > > > > > > > > > > > > > > > > > > > > > > > > > ---- > > > > > > > > > > > > > > > > > > > > > > > > > > > > All of that being said, I am > absolutely > > > > happy > > > > > > to > > > > > > > > > agree > > > > > > > > > > to > > > > > > > > > > > disagree, I > > > > > > > > > > > > > think > > > > > > > > > > > > > > to a certain extent this is down to a > > > > > question > > > > > > of > > > > > > > > > > personal > > > > > > > > > > > > > > style/preference. And as this is your > > > baby > > > > > and > > > > > > > you > > > > > > > > > > have put > > > > > > > > > > > a lot more > > > > > > > > > > > > > > effort and thought into it than I > ever > > > will > > > > > > I'll > > > > > > > > shut > > > > > > > > > > up now > > > > > > > > > > > :) > > > > > > > > > > > > > > > > > > > > > > > > > > > > Again, thanks for all your good work! > > > > > > > > > > > > > > > > > > > > > > > > > > > > Best regards, > > > > > > > > > > > > > > Sönke > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Fri, Nov 30, 2018 at 9:00 PM > Ryanne > > > > Dolan > > > > > < > > > > > > > > > > > ryannedo...@gmail.com> > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Thanks Sönke. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > it just feels to me like an awful > > lot > > > > of > > > > > > > > Connect > > > > > > > > > > > functionality > > > > > > > > > > > > would > > > > > > > > > > > > > > need > > > > > > > > > > > > > > > to be reimplemented or at least > > wrapped > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Connect currently has two drivers, > > > > > > > > > > ConnectDistributed and > > > > > > > > > > > > > > > ConnectStandalone. Both set up a > > > Herder, > > > > > > which > > > > > > > > > > manages > > > > > > > > > > > Workers. I've > > > > > > > > > > > > > > > implemented a third driver which > sets > > > up > > > > > > > multiple > > > > > > > > > > Herders, > > > > > > > > > > > one for > > > > > > > > > > > > each > > > > > > > > > > > > > > > Kafka cluster as specified in a > > config > > > > > file. > > > > > > > From > > > > > > > > > the > > > > > > > > > > > Herder level > > > > > > > > > > > > > down, > > > > > > > > > > > > > > > nothing is changed or duplicated -- > > > it's > > > > > just > > > > > > > > > > Connect. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > For the REST API, Connect wraps a > > > Herder > > > > > in a > > > > > > > > > > RestServer > > > > > > > > > > > class, which > > > > > > > > > > > > > > > creates a Jetty server with a few > > > JAX-RS > > > > > > > > resources. > > > > > > > > > > One of > > > > > > > > > > > these > > > > > > > > > > > > > > resources > > > > > > > > > > > > > > > is ConnectorsResource, which is the > > > real > > > > > meat > > > > > > > of > > > > > > > > > the > > > > > > > > > > REST > > > > > > > > > > > API, > > > > > > > > > > > > enabling > > > > > > > > > > > > > > > start, stop, creation, deletion, > and > > > > > > > > configuration > > > > > > > > > of > > > > > > > > > > > Connectors. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > I've added MirrorRestServer, which > > > wraps > > > > a > > > > > > set > > > > > > > of > > > > > > > > > > Herders > > > > > > > > > > > instead of > > > > > > > > > > > > > one. > > > > > > > > > > > > > > > The server exposes a single > resource, > > > > > > > > > > ClustersResource, > > > > > > > > > > > which is > > > > > > > > > > > > only a > > > > > > > > > > > > > > few > > > > > > > > > > > > > > > lines of code: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > @GET > > > > > > > > > > > > > > > @Path("/") > > > > > > > > > > > > > > > public Collection<String> > > > listClusters() > > > > { > > > > > > > > > > > > > > > return clusters.keySet(); > > > > > > > > > > > > > > > } > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > @Path("/{cluster}") > > > > > > > > > > > > > > > public ConnectorsResource > > > > > > > > > > > > > > getConnectorsForCluster(@PathParam("cluster") > > > > > > > > > > > > > > > cluster) { > > > > > > > > > > > > > > > return new > > > > > > > > > > ConnectorsResource(clusters.get(cluster)); > > > > > > > > > > > > > > > } > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > (simplified a bit and subject to > > > change) > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > The ClustersResource defers to the > > > > existing > > > > > > > > > > > ConnectorsResource, which > > > > > > > > > > > > > > again > > > > > > > > > > > > > > > is most of the Connect API. With > this > > > in > > > > > > > place, I > > > > > > > > > > can make > > > > > > > > > > > requests > > > > > > > > > > > > > like: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > GET /clusters > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > GET /clusters/us-west/connectors > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > PUT > > > > > > /clusters/us