The CEP for the sidecar has stalled. The sidecar itself is very much alive and a thing.
CEP != artifact. We should definitely clean that up though. On Mon, Sep 30, 2024, at 10:59 AM, Dinesh Joshi wrote: > Patrick, could you please elaborate? The Sidecar has been a thing for a while > now. > > On Mon, Sep 30, 2024 at 7:51 AM Patrick McFadin <pmcfa...@gmail.com> wrote: >> I made the mistake of asking two things in one email. >> >> First thing I asked. Sidecar? Stalled CEP so why is this being talked about >> like this is a thing? >> >> On Mon, Sep 30, 2024 at 7:21 AM Benedict <bened...@apache.org> wrote: >>> >>> Sorry Bernardo, you may have misunderstood me. I don’t have any concerns, I >>> was suggesting a possible future scenario where CDC for Kafka via sidecar >>> is changed to use a hypothetical future topic subscription service provided >>> by C*. It was meant to show that this CEP may be easily decoupled from any >>> future evolution in this area. >>> >>> >>>> On 30 Sep 2024, at 14:58, Bernardo Botella <conta...@bernardobotella.com> >>>> wrote: >>>> Thanks everyone for the comments. >>>> >>>> Patrick: >>>> The proposal includes a “best effort” approach for deduplication (some >>>> details can be found on the Digest class comments on the PR here >>>> https://github.com/apache/cassandra-analytics/pull/87/files#diff-3a09caecc1da13419d92cde56a7cfc7d253faac08182e6c2768b3d32c015de82R185-R193 >>>> ). That alone won’t eliminate all the duplicates, but as Josh points out, >>>> it moves the line to something way easier to handle for consumers, and >>>> definitely on the direction we should aim towards. Accord is definitely >>>> something this contribution will benefit from, that will move that line >>>> even further. >>>> >>>> Benedict: >>>> If I understand it correctly, your concern is that Kafka is somewhat the >>>> hardcoded option for a CDC stream being published? The proposal introduces >>>> a concept of data sources and sinks >>>> (https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=323488575#CEP44:KafkaintegrationforCassandraCDCusingSidecar-SourcesandSinks) >>>> being kafka the first implemented data sink. That means that the actual >>>> Kafka output should (will) be something pluggable. >>>> >>>> >>>> >>>>> On Sep 30, 2024, at 5:43 AM, Josh McKenzie <jmcken...@apache.org> wrote: >>>>> >>>>>> I don't see much on how this would be handled other than "left to the >>>>>> end user to figure out." >>>>> My immediate thought when I read that was "Yes. But it's moving where we >>>>> draw the line of 'left to the end user to figure out' *much further* than >>>>> it was before". >>>>> >>>>> This should only be necessary in edge cases w/extended severe degraded >>>>> availability where you can't hit QUORUM w/this design. So we go from >>>>> "De-dupe literally everything o ye' user" to "de-dupe a small fraction of >>>>> a % of the time when things really go off the rails". >>>>> >>>>> It still leaves the burden of processing potential duplicates downstream, >>>>> so some *complexity* burden on the users remains if they have no >>>>> tolerance for processing duplicate messages, however the underlying >>>>> machine resource utilization (from "dedupe everything" to "dedupe a small >>>>> % of things") is pretty massively shifted by this design change. That, >>>>> and using the hash of the mutation the way the extended design does is >>>>> something a downstream consumer could also do on their side to ensure >>>>> anything that came in past the drop-off window wasn't already seen. So >>>>> not *too* painful; certainly a vast improvement over the status quo. >>>>> >>>>> As to TCM and Accord: absolutely agree. I'd love to see a world where we >>>>> Accord everything and fire off CDC to subscribers from a coordinator >>>>> bypassing all this LSM-bastardized post-processing for CDC for instance. >>>>> That said, this is a functionality users needed back in... 2016? When we >>>>> first added CDC. So I think it's worth it to move on it now while >>>>> retaining architectural options to move to updated metadata and >>>>> transactions as they mature (obviously we'll lean on TCM since it's in >>>>> 5.0 / trunk right now; more applies to the accord bit). >>>>> >>>>> On Mon, Sep 30, 2024, at 3:20 AM, Benedict wrote: >>>>>> >>>>>> Yes, with accord it should be fairly easy to have reliable no-dupe log >>>>>> streaming without an elected leader. Given the broad set of use cases, I >>>>>> can imagine supporting some more native topic subscription API in C* >>>>>> rather than requiring Kafka, so perhaps any integration of Kafka with >>>>>> the sidecar can be considered a separate parallel effort, that might >>>>>> eventually implement itself with this C* feature whenever it >>>>>> materialises? >>>>>> >>>>>> >>>>>>> On 30 Sep 2024, at 03:42, Jeff Jirsa <jji...@gmail.com> wrote: >>>>>>> >>>>>>> >>>>>>> Transactional metadata and Accord should make it MUCH easier to do >>>>>>> duplication avoiding CDC (and I was going to note that someone should >>>>>>> ensure that the interfaces exposed to the public are stable enough not >>>>>>> to change the published api once those exist) >>>>>>> >>>>>>> >>>>>>> >>>>>>>> On Sep 29, 2024, at 7:04 PM, Patrick McFadin <pmcfa...@gmail.com> >>>>>>>> wrote: >>>>>>>> >>>>>>>> As I was reviewing this, it occurred to me that it was talking about >>>>>>>> Sidecar like it was a thing but that CEP has been stalled for quite >>>>>>>> some time: >>>>>>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=95652224 >>>>>>>> >>>>>>>> If work on this is being done, should we get this official and wrapped >>>>>>>> up? >>>>>>>> >>>>>>>> On to the proposal... >>>>>>>> >>>>>>>> This has been a topic on the project for over 10 years now. I've seen >>>>>>>> multiple goes at making this work and the issue that always turns out >>>>>>>> to torpedo the project is handing dupes. To the point that they go >>>>>>>> from a generalized Kafka producer engine to something specific to a >>>>>>>> particular use case. I don't see much on how this would be handled >>>>>>>> other than "left to the end user to figure out." >>>>>>>> >>>>>>>> There is also little mention of where the increased resource load >>>>>>>> would be handled. >>>>>>>> >>>>>>>> This has been discussed many times before, but is it time to introduce >>>>>>>> the concept of an elected leader for a token range for this type of >>>>>>>> operation? It would eliminate a ton of problems that need to managed >>>>>>>> when bridging c* to a system like Kafka. Last time it was discussed in >>>>>>>> earnest was for KIP-30: >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-30+-+Allow+for+brokers+to+have+plug-able+consensus+and+meta+data+storage+sub+systems >>>>>>>> >>>>>>>> >>>>>>>> Patrick >>>>>>>> >>>>>>>> On Sat, Sep 28, 2024 at 11:44 AM Jon Haddad <j...@rustyrazorblade.com> >>>>>>>> wrote: >>>>>>>>> Yes! I’m really looking forward to trying this out. The CEP looks >>>>>>>>> really well thought out. I think this will make CDC a lot more useful >>>>>>>>> for a lot of teams. >>>>>>>>> Jon >>>>>>>>> >>>>>>>>> >>>>>>>>> On Fri, Sep 27, 2024 at 4:23 PM Josh McKenzie <jmcken...@apache.org> >>>>>>>>> wrote: >>>>>>>>>> __ >>>>>>>>>> Really excited to see this hit the ML James. >>>>>>>>>> >>>>>>>>>> As author of the base CDC (get your stones ready for throwing :D) >>>>>>>>>> and someone moderately involved in the CEP here, definitely welcome >>>>>>>>>> any questions. CDC is a *thorny* *problem *in a multi-replica >>>>>>>>>> distributed system like this. >>>>>>>>>> >>>>>>>>>> On Fri, Sep 27, 2024, at 5:40 PM, James Berragan wrote: >>>>>>>>>>> Hi everyone, >>>>>>>>>>> >>>>>>>>>>> Wiki: >>>>>>>>>>> https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-44%3A+Kafka+integration+for+Cassandra+CDC+using+Sidecar >>>>>>>>>>> >>>>>>>>>>> We would like to propose this CEP for adoption by the community. >>>>>>>>>>> >>>>>>>>>>> CDC is a common technique in databases but right now there is no >>>>>>>>>>> out-of-the-box solution to do this easily and at scale with >>>>>>>>>>> Cassandra. Our proposal is to build a fully-fledged solution into >>>>>>>>>>> the Apache Cassandra Sidecar. This comes with a number of benefits: >>>>>>>>>>> - Sidecar is an official part of the existing Cassandra eco-system. >>>>>>>>>>> - Sidecar runs co-located with Cassandra instances and so scales >>>>>>>>>>> with the cluster size. >>>>>>>>>>> - Sidecar can access the underlying Cassandra database to store CDC >>>>>>>>>>> configuration and the CDC state in a special table. >>>>>>>>>>> - Running in the Sidecar does not require additional external >>>>>>>>>>> resources to run. >>>>>>>>>>> >>>>>>>>>>> The core CDC module we anticipate will be pluggable and re-usable, >>>>>>>>>>> it is available for review here: >>>>>>>>>>> https://github.com/apache/cassandra-analytics/pull/87. The >>>>>>>>>>> remaining Sidecar code will follow. >>>>>>>>>>> >>>>>>>>>>> As a reminder, please keep the discussion here on the dev list vs. >>>>>>>>>>> in the wiki, as we’ve found it easier to manage via email. >>>>>>>>>>> >>>>>>>>>>> Sincerely, >>>>>>>>>>> James Berragan >>>>>>>>>>> Bernardo Botella Corbi >>>>>>>>>>> Yifan Cai >>>>>>>>>>> Jyothsna Konisa