Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Store user defined insert_id in zookeper #7461

Closed
laskoviymishka opened this issue Oct 23, 2019 · 2 comments · Fixed by #32304
Closed

Store user defined insert_id in zookeper #7461

laskoviymishka opened this issue Oct 23, 2019 · 2 comments · Fixed by #32304
Labels
comp-replication Replicated tables easy task Good for first contributors feature st-accepted The issue is in our backlog, ready to take

Comments

@laskoviymishka
Copy link

Use case

I want to gurantee exactly once delivery to clickhouse without requiring separate persistence for replicated merge tree families when i use upload from some message broker (kafka-like).
Most message brokers has stream api. We read buffer of data, push it to clickhouse and commit offsets when it done. So in case of failures such client code may experience double read. If worker push data to clikchouse it may lead to double writes aswell. Clickhouse already has well defined deduplication mechanism, clickhouse will dedublicate data if you push exact same batch multiple time. So in our worker (assuming we write in single thread to CH) we must know last request offsets. To achive these i have to use separate storage and do 2 writes into it (before insert with right offset, and after ack from CH to confirm left offset).

User flow looks something like this:

  1. Read data from broker (let say 1000 record, from N to M offset)
  2. Store right ofset to M
  3. Start upload data to clickhouse
  4. Die in agony
  5. Restore worker, look to offset store, and see that last time there was N to M offset sent
  6. Read from broker exact same offsets
  7. Push it again to clickhouse
  8. Store M offset to left.

Describe the solution you'd like

Main idea to eliminate need of separate storage. Right now clickhouse already utilize persistence via ZK, so we could use it for these need. If user will provide some meaningfull (for him) string with each write request we can consistent store it to ZK. Then on client side we may look at it and make a decision of what offset may be skipped.

Ideal user flow looks something like this:

  1. Read data from broker from N to M.
  2. Start upload to clickhouse with insert id N_M
  3. Die with happy smile
  4. Restore worker. Look to clickhosue to latest insert id (N_M recieved)
  5. Skip offset till M
  6. Go to 1

Describe alternatives you've considered
We could continue to use separate offset store.

Additional context
It may be also usefull for internal use of kafka storage (wich also have same proble).

@alexey-milovidov alexey-milovidov added the comp-replication Replicated tables label Jun 15, 2020
@alexey-milovidov alexey-milovidov added st-accepted The issue is in our backlog, ready to take warmup task The task for new ClickHouse team members. Low risk, moderate complexity, no urgency. labels Jul 3, 2021
@alexey-milovidov
Copy link
Member

Makes sense.

We can introduce a setting named insert_deduplication_token with String value.
It can be provided by user with unique value on every INSERT.
If it is non empty, we will hash it along with the partition id and the ordinal number of block if data was split to multiple blocks,
and use this hash instead of a digest of data.

Looks easy to implement.

@alexey-milovidov alexey-milovidov added easy task Good for first contributors and removed warmup task The task for new ClickHouse team members. Low risk, moderate complexity, no urgency. labels Jul 3, 2021
@devcrafter
Copy link
Member

Ideal user flow looks something like this:
...
4. ... Look to clickhosue to latest insert id (N_M recieved)

Please, could you elaborate here - how exactly is the user supposed to do this?

devcrafter added a commit to devcrafter/ClickHouse that referenced this issue Nov 21, 2021
The setting allows a user to provide own deduplication semantic in Replicated*MergeTree
If provided, it's used instead of data digest to generate block ID
So, for example, by providing a unique value for the setting in each INSERT statement,
user can avoid the same inserted data being deduplicated

Issue: ClickHouse#7461
devcrafter added a commit to devcrafter/ClickHouse that referenced this issue Dec 19, 2021
The setting allows a user to provide own deduplication semantic in Replicated*MergeTree
If provided, it's used instead of data digest to generate block ID
So, for example, by providing a unique value for the setting in each INSERT statement,
user can avoid the same inserted data being deduplicated

Inserting data within the same INSERT statement are split into blocks
according to the *insert_block_size* settings
(max_insert_block_size, min_insert_block_size_rows, min_insert_block_size_bytes).
Each block with the same INSERT statement will get an ordinal number.
The ordinal number is added to insert_deduplication_token to get block dedup token
i.e. <token>_0, <token>_1, ... Deduplication is done per block
So, to guarantee deduplication for two same INSERT queries,
dedup token and number of blocks to have to be the same

Issue: ClickHouse#7461
devcrafter referenced this issue in devcrafter/ClickHouse Feb 1, 2022
+ reduce number of allocations on replication merge tree path
+ bash test: move insert block settings into variable

Issue: ClickHouse#7461
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
comp-replication Replicated tables easy task Good for first contributors feature st-accepted The issue is in our backlog, ready to take
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants