--
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/cc539d94-d4ad-467b-ac12-74f888827a99n%40googlegroups.com.
Hi Jiri,Thank you for sharing this! I am really excited to see this functionality added to Debezium as large-table snapshotting is the only real pain point I've experienced running DBZ.I've read through the design doc. For the open question of How to avoid a never-ending incremental snapshot if there are inserts happening continously? I am curious to know if this is an issue only when the rate of inserts is greater than the rate of snapshotting? And if it is, I wonder if there is a viable solution other than increasing the rate of the snapshotting behaviour?
For the second open question How to deal with different PK types? And composite PKs?, are there cases where a primary key/composite key wouldn't allow sorting deterministically?
Best wishes,Philip
--Hi,please take a look at https://github.com/debezium/debezium-design-documents/blob/main/DDD-3.mdIn this document we propose implementation of two features1) Ad-hoc snapshotting that would allow the user to trigger a snapshot of table(s) anytime during the streaming2) Incremental snapshotting that would enable the user to take the snapshot of a table during the streaming (this is an off-spring of DBLog implementation)Please take a look at the proposal, any comments are welcome and would be incoirporated either directly in the first cur or in a follow-up steps.Thanks in advance!Jiri Pechanec--
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/cc539d94-d4ad-467b-ac12-74f888827a99n%40googlegroups.com.
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/CAFZ8o0O1QEXPAmbCemk0Bgsc%3D-WEgJLb8XHDTE9XecFL-4tNOQ%40mail.gmail.com.
Hi,
I have a few questions/thoughts about the DDD-3:
1. How to handle schema changes during the incremental snapshot?
Will it be possible to detect the schema change from the binlog stream and re-snapshot the schemas for the next chunk?
What if the schema change affects PK, will the incremental snapshot fail or restart?
2. How to scope the snapshot to all tables/list of tables/a range of primary keys/maybe even a specific query?
3. How can the snapshotting be paused and resumed without pausing the binlog stream from the connector?
4. Where the incremental snapshotting progress will be stored (Kafka topic, Zookeeper or something else)?
5. What will be the format of the incremental snapshotting progress log?
6. Will the incremental snapshotting resume from where it stopped if the connector gets restarted?
7. DDD-3 says parallelization of snapshotting is a non-goal. Could the parallelization be added later or there are some blockers for it in the current design? If there are no blocker, what needs to be done to make it simpler to add it in the future (id for each chunk, low and high watermark containing the chunk id)?
8. Is it possible to send low and high watermark to the binlog stream without having to write to a database, so that Debezium would be able to keep being connected to a read-only replica?
--KateHi,
I have a few questions/thoughts about the DDD-3:
1. How to handle schema changes during the incremental snapshot?
Will it be possible to detect the schema change from the binlog stream and re-snapshot the schemas for the next chunk?
What if the schema change affects PK, will the incremental snapshot fail or restart?
2. How to scope the snapshot to all tables/list of tables/a range of primary keys/maybe even a specific query?
3. How can the snapshotting be paused and resumed without pausing the binlog stream from the connector?
4. Where the incremental snapshotting progress will be stored (Kafka topic, Zookeeper or something else)?
5. What will be the format of the incremental snapshotting progress log?
6. Will the incremental snapshotting resume from where it stopped if the connector gets restarted?
7. DDD-3 says parallelization of snapshotting is a non-goal. Could the parallelization be added later or there are some blockers for it in the current design? If there are no blocker, what needs to be done to make it simpler to add it in the future (id for each chunk, low and high watermark containing the chunk id)?
8. Is it possible to send low and high watermark to the binlog stream without having to write to a database, so that Debezium would be able to keep being connected to a read-only replica?
>> 2. How to scope the snapshot to all tables/list of tables
> Currently you can enumerate the tables you want to snapshot. Snapshotting of ALL tables would require more effort as we would need to monitor if there is a new table appearing during the snapshot.
Sorry, my question sounds confusing, what I meant by all tables is the list of the tables the connector is reading binlogs from. For example, the connector is configured to read binlogs from 10 tables. Will I be able to incrementally snapshot only two of those tables while still getting the binlogs from the rest of the 8 tables too?
>> 5. What will be the format of the incremental snapshotting progress log?
> I am not sure I understand the question. Do you mean application logging or storage logging?
This was a follow-up question to 4. Where the incremental snapshotting progress will be stored (Kafka topic, Zookeeper or something else)?
What will be the content of the incremental snapshotting progress message? How to make it flexible enough to support the incremental snapshotting parallelization in the future?
The format will be different from Kafka Connect Offsets, will it break anything?You received this message because you are subscribed to a topic in the Google Groups "debezium" group.
To unsubscribe from this topic, visit https://groups.google.com/d/topic/debezium/JS3LjfE3WQo/unsubscribe.
To unsubscribe from this group and all its topics, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/7410bec7-bc32-4f52-8573-0866ec593976n%40googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/33d876f4-fd3c-4e85-8b0a-ae827840e7f8n%40googlegroups.com.