-
Notifications
You must be signed in to change notification settings - Fork 4
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
Implement VersionedKeyValueStore<K, V> #21
Labels
Comments
hartmut-co-uk
added
documentation
Improvements or additions to documentation
enhancement
New feature or request
labels
Jul 9, 2023
Initial design... Draft table schema for
|
Here's the scratch file from the live coding example (https://youtu.be/zuMvGdmRqfs) CREATE KEYSPACE IF NOT EXISTS "test" WITH replication = {'class':'SimpleStrategy', 'replication_factor' : 1};
USE test;
CREATE TABLE IF NOT EXISTS word_count_kstreams_store
(
partition int,
key blob,
validFrom timestamp,
validTo timestamp,
time timestamp,
value blob,
PRIMARY KEY ((partition), key, validTo)
) WITH compaction = { 'class' : 'LeveledCompactionStrategy' };
-- ) WITH CLUSTERING ORDER BY (key ASC, validTo ASC) AND compaction = { 'class' : 'LeveledCompactionStrategy' };
-- DONE: TBC should clustering order use `validTo ASC`? -> decision: use ASC, since that's what we need to use for our current known data access patterns
CREATE TABLE IF NOT EXISTS clicks_global_kstreams_store
(
key blob,
validFrom timestamp,
validTo timestamp,
time timestamp,
value blob,
PRIMARY KEY ((key), validTo)
) WITH compaction = { 'class' : 'LeveledCompactionStrategy' };
-- ) WITH CLUSTERING ORDER BY (validTo ASC) AND compaction = { 'class' : 'LeveledCompactionStrategy' };
-- DONE: TBC should clustering order use `validTo ASC`? -> decision: use ASC, since that's what we need to use for our current known data access patterns
SELECT * FROM word_count_kstreams_store;
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 0, 4000, 4000, intAsBlob(0));
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 4000, 8000, 8000, intAsBlob(4));
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 8000, 10000, 10000, intAsBlob(8));
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 10000, -1, 60000, intAsBlob(10));
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store;
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo=-1;
-- current is validFrom 10000
-- WRONG: SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo<=5000 AND validTo>=0;
-- note: `validTo<=` is unnecessary, see following line .. SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo<=10000 AND validTo>=5000 ORDER BY validTo ASC LIMIT 1;
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo>=5000 ORDER BY validTo ASC LIMIT 1;
-- new current -> insert at time 15, price 15$, at stream time 15
-- step 1: validate time is within the retention window (30s) to current stream time 15 - true
-- step 2: get current value
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo=-1;
-- current is validFrom 10000
-- step 3: new record is after current - true
-- step 4: (logical) update prev. record & insert new record
-- (actual cql) upsert current (update value+validFrom) - insert the prev. record as new row
BEGIN BATCH
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 15000, -1, 15000, intAsBlob(15));
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 10000, 15000, 10000, intAsBlob(10));
APPLY BATCH;
-- new late record -> insert at time 12, price 12$, at stream time 16
-- step 1: validate time is within the retention window (30s) to current stream time 16 - true
-- step 2: get current value
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo=-1;
-- current is validFrom 15000
-- step 3: new record is after current - false
-- step 4: fetch record `asOfTimestamp` 12
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo>=12000 ORDER BY validTo ASC LIMIT 1;
-- record 10$ valid 10...15
-- step 5: update prev. record & insert new record
BEGIN BATCH
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 10000, 12000, 10000, intAsBlob(10));
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 12000, 15000, 16000, intAsBlob(12));
APPLY BATCH;
-- test... asOfTime 13
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo>=13000 ORDER BY validTo ASC LIMIT 1;
-- assert 12$
-- delete value at time 13, at stream time 17
-- step 1: validate time is within the retention window (30s) to current stream time 17 - true
-- step 2: fetch current record and check if validFrom<=13
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo=-1;
-- current is validFrom 15000
-- step 3: fetch record `asOfTimestamp` 13
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo>=13000 ORDER BY validTo ASC LIMIT 1;
-- record 12$ valid 12...15
-- step 4: fetch previous record `asOfTimestamp` 12 (the `validFrom` of the record)
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo>=12000 ORDER BY validTo ASC LIMIT 1;
-- record 10$ valid 10...12
-- step 5: delete versioned record, delete prev. versioned record, insert 'new previous'
BEGIN BATCH
-- not required, cause we're going to upsert anyway... DELETE FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo=15000;
DELETE FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo=12000;
INSERT INTO word_count_kstreams_store (partition, key, validFrom, validTo, time, value) VALUES (0, textAsBlob('curry'), 10000, 15000, 10000, intAsBlob(10));
APPLY BATCH;
-- 1691235573402988
-- test... asOfTime 13
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo>=13000 ORDER BY validTo ASC LIMIT 1;
-- assert: 10$
-- all versioned records for key
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry');
-- all versioned records for key, before validTo 6
SELECT partition, blobAsText(key), blobAsInt(value), validFrom, validTo, time FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo<=6000 AND validTo>=0;
-- cleanup in reg. to retention time
-- step 1: apply range delete
DELETE FROM word_count_kstreams_store WHERE partition=0 AND key=textAsBlob('curry') AND validTo<=9000 AND validTo>=0;
-- test example from Kafka Summit London 2023 Talk (slide 13) with versioned global store
SELECT * FROM clicks_global_kstreams_store;
-- inserts
INSERT INTO clicks_global_kstreams_store (key, validFrom, validTo, time, value) VALUES (textAsBlob('curry'), 0, 4000, 4000, intAsBlob(8));
INSERT INTO clicks_global_kstreams_store (key, validFrom, validTo, time, value) VALUES (textAsBlob('curry'), 4000, 17000, 8000, intAsBlob(10));
INSERT INTO clicks_global_kstreams_store (key, validFrom, validTo, time, value) VALUES (textAsBlob('curry'), 17000, 38000, 10000, intAsBlob(12));
INSERT INTO clicks_global_kstreams_store (key, validFrom, validTo, time, value) VALUES (textAsBlob('curry'), 38000, 53000, 10000, intAsBlob(11));
INSERT INTO clicks_global_kstreams_store (key, validFrom, validTo, time, value) VALUES (textAsBlob('curry'), 53000, -1, 60000, intAsBlob(9));
-- 30s retention time 'cleanup' applied at time '53
DELETE FROM clicks_global_kstreams_store WHERE key=textAsBlob('curry') AND validTo<=23000 AND validTo>=0;
-- select examples
SELECT blobAsText(key), blobAsInt(value), validFrom, validTo FROM clicks_global_kstreams_store WHERE key=textAsBlob('curry');
SELECT blobAsText(key), blobAsInt(value), validFrom, validTo FROM clicks_global_kstreams_store WHERE key=textAsBlob('curry') AND validTo=-1;
SELECT blobAsText(key), blobAsInt(value), validFrom, validTo FROM clicks_global_kstreams_store WHERE key=textAsBlob('curry') AND validTo>=50000 ORDER BY validTo ASC LIMIT 1;
SELECT blobAsText(key), blobAsInt(value), validFrom, validTo FROM clicks_global_kstreams_store WHERE key=textAsBlob('curry') AND validTo>=35000 ORDER BY validTo ASC LIMIT 1;
SELECT blobAsText(key), blobAsInt(value), validFrom, validTo FROM clicks_global_kstreams_store WHERE key=textAsBlob('curry') AND validTo>=30000 ORDER BY validTo ASC LIMIT 1; |
hartmut-co-uk
added a commit
that referenced
this issue
Aug 13, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 13, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 13, 2023
- (for versioned store) - covers edge cases
hartmut-co-uk
added a commit
that referenced
this issue
Aug 19, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 19, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 19, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 19, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 19, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 21, 2023
hartmut-co-uk
added a commit
that referenced
this issue
Aug 26, 2023
completed with #27 |
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Labels
Support
VersionedKeyValueStore<K, V>
Shipped with Kafka 3.5.0.
ref KIP-889: Versioned State Stores
Talk at Kafka Summit London 2023:
https://www.confluent.io/events/kafka-summit-london-2023/versioned-state-stores-in-kafka-streams/
Motivation
(quoted from KIP-889)
Versioned Store Interface
The text was updated successfully, but these errors were encountered: