Jan's response is correct, but I'd like to emphasize the impact on a Flink
application.

If the compaction happens before the data arrives in Flink, the
intermediate updates are lost and just the final result appears.
Also if you restart your Flink application and reprocess older data, it
will naturally only see the compacted data save for the active segment.

So how to make it deterministic? Simply drop topic compaction. If it's
coming from CDC and you want to process and produce changelog streams over
several applications, you probably don't want to use log compactions
anyways.

Log compaction only makes sense in the snapshot topic that displays the
current state (KTable), where you don't think in CDC updates anymore but
just final records, like
(user_id: 1, state: "california")
(user_id: 1, state: "ohio")

Usually, if you use CDC in your company, each application is responsible
for building its own current model by tapping in the relevant changes. Log
compacted topics would then only appear at the end of processing, when you
hand it over towards non-analytical applications, such as Web Apps.

On Wed, Feb 24, 2021 at 10:01 AM Jan Lukavský <je...@seznam.cz> wrote:

> Hi Rex,
>
> If I understand correctly, you are concerned about behavior of Kafka
> source in the case of compacted topic, right? If that is the case, then
> this is not directly related to Flink, Flink will expose the behavior
> defined by Kafka. You can read about it for instance here [1]. TL;TD - your
> pipeline is guaranteed to see every record written to topic (every single
> update, be it later "overwritten" or not) if it processes the record with
> latency at most 'delete.retention.ms'. This is configurable per topic -
> default 24 hours. If you want to reprocess the data later, your consumer
> might see only resulting compacted ("retracted") stream, and not every
> record actually written to the topic.
>
>  Jan
>
> [1]
> https://medium.com/swlh/introduction-to-topic-log-compaction-in-apache-kafka-3e4d4afd2262
> On 2/24/21 3:14 AM, Rex Fenley wrote:
>
> Apologies, forgot to finish. If the Kafka source performs its own
> retractions of old data on key (user_id) for every append it receives, it
> should resolve this discrepancy I think.
>
> Again, is this true? Anything else I'm missing?
>
> Thanks!
>
>
> On Tue, Feb 23, 2021 at 6:12 PM Rex Fenley <r...@remind101.com> wrote:
>
>> Hi,
>>
>> I'm concerned about the impacts of Kafka's compactions when sending data
>> between running flink jobs.
>>
>> For example, one job produces retract stream records in sequence of
>> (false, (user_id: 1, state: "california") -- retract
>> (true, (user_id: 1, state: "ohio")) -- append
>> Which is consumed by Kafka and keyed by user_id, this could end up
>> compacting to just
>> (true, (user_id: 1, state: "ohio")) -- append
>> If some other downstream Flink job has a filter on state == "california"
>> and reads from the Kafka stream, I assume it will miss the retract message
>> altogether and produce incorrect results.
>>
>> Is this true? How do we prevent this from happening? We need to use
>> compaction since all our jobs are based on CDC and we can't just drop data
>> after x number of days.
>>
>> Thanks
>>
>> --
>>
>> Rex Fenley  |  Software Engineer - Mobile and Backend
>>
>>
>> Remind.com <https://www.remind.com/> |  BLOG <http://blog.remind.com/>
>>  |  FOLLOW US <https://twitter.com/remindhq>  |  LIKE US
>> <https://www.facebook.com/remindhq>
>>
>
>
> --
>
> Rex Fenley  |  Software Engineer - Mobile and Backend
>
>
> Remind.com <https://www.remind.com/> |  BLOG <http://blog.remind.com/>  |
>  FOLLOW US <https://twitter.com/remindhq>  |  LIKE US
> <https://www.facebook.com/remindhq>
>
>

Reply via email to