Hi Ravi,

Thanks for the approval and the questions, please find the comments below

1. Generally CDC includes IUD operations, so how are you planning to handle
them? Are you planning to merge command? If yes how frequently do you want
to
merge it?
You are right, it includes mainly the IUD operations. So I will be
preparing the data frames by reading data from the source(Kafka or DFS) and
then call
merge APIs which are introduced in the last carbondata 2.2.0 release.
A configuration or the CLI argument will be exposed to the user, he can
decide the batch time interval. If not given we will use the default which
we need to decide
 5 or 10 seconds based on the delay and the processing and data rate.

2. How you can make sure the Kafka exactly-once semantics( how can you
ensure data is written once without duplication)
We have exposed two properties for the deduplication, one is for insert and
one is for upsert.
During insert, if the property is enabled the duplicate records which are
already present in the target table will be dropped.
In an upsert case, the duplicates(just in the incoming dataset) will be
removed based on the latest record.

In the scenario of the same batch comes for upsert again, in that case,
carbon doesn't write any duplicate data, it will just do a reupdate, which
inturn doesn't affect the store.
Here better to make one more update operation, instead of maintaining some
metadata to identify whether this batch has been processed before or not.

Please give your suggestion on this.

Thanks

Regards,
Akash




On Wed, Sep 1, 2021 at 6:11 PM Ravindra Pesala <ravi.pes...@gmail.com>
wrote:

> +1
>
> I want to understand few clarifications regarding the design.
> 1. Generally CDC includes IUD operations, so how are you planning to handle
> them? Are you planning to merge command? If yes how frequent you want to
> merge it?
> 2. How you can make sure the Kafka exactly once semantics( how can you
> ensure data is written once with out duplication)
>
>
> Regards,
> Ravindra.
>
> On Wed, 1 Sep 2021 at 1:48 AM, Akash Nilugal <akashnilu...@gmail.com>
> wrote:
>
> > Hi Community,
> >
> > OLTP systems like Mysql are used heavily for storing transactional data
> in
> > real-time and the same data is later used for doing fraud detection and
> > taking various data-driven business decisions. Since OLTP systems are not
> > suited for analytical queries due to their row-based storage, there is a
> > need to store this primary data into big data storage in a way that data
> on
> > DFS is an exact replica of the data present in Mysql. Traditional ways
> for
> > capturing data from primary databases, like Apache Sqoop, use pull-based
> > CDC approaches which put additional load on the primary databases. Hence
> > log-based CDC solutions became increasingly popular. However, there are 2
> > aspects to this problem. We should be able to incrementally capture the
> > data changes from primary databases and should be able to incrementally
> > ingest the same in the data lake so that the overall latency decreases.
> The
> > former is taken care of using log-based CDC systems like Maxwell and
> > Debezium. Here we are proposing a solution for the second aspect using
> > Apache Carbondata.
> >
> > Carbondata streamer tool enables users to incrementally ingest data from
> > various sources, like Kafka and DFS into their data lakes. The tool comes
> > with out-of-the-box support for almost all types of schema evolution use
> > cases. Currently, this tool can be launched as a spark application either
> > in continuous mode or a one-time job.
> >
> > Further details are present in the design document. Please review the
> > design and help to improve it. I'm attaching the link to the google doc,
> > you can directly comment on that. Any suggestions and improvements are
> most
> > welcome.
> >
> >
> >
> https://docs.google.com/document/d/1x66X5LU5silp4wLzjxx2Hxmt78gFRLF_8IocapoXxJk/edit?usp=sharing
> >
> > Thanks
> >
> > Regards,
> > Akash R Nilugal
> >
> --
> Thanks & Regards,
> Ravi
>

Reply via email to