Skip to content

Conversation

@giuseppelillo
Copy link
Contributor

@giuseppelillo giuseppelillo commented Jan 15, 2026

Introduces control plane support for initializing diskless logs, enabling migrating topics from classic to diskless. The initialization transfers log offsets and producer state.

@giuseppelillo giuseppelillo changed the base branch from topic-type-switcher to main January 16, 2026 13:25
@giuseppelillo giuseppelillo changed the base branch from main to topic-type-switcher January 16, 2026 13:39
@giuseppelillo giuseppelillo changed the base branch from topic-type-switcher to main January 19, 2026 09:35
@giuseppelillo giuseppelillo force-pushed the giuseppelillo/init-diskless-log branch from 3827745 to 79bea61 Compare January 19, 2026 11:40
@giuseppelillo giuseppelillo changed the title Giuseppelillo/init diskless log Add control plane method for initializing a diskless log that does not start from 0 Jan 19, 2026
@giuseppelillo giuseppelillo marked this pull request as ready for review January 19, 2026 12:59
@giuseppelillo giuseppelillo changed the title Add control plane method for initializing a diskless log that does not start from 0 Control Plane support for initializing the diskless log Jan 19, 2026
Introduces control plane support for initializing diskless logs,
enabling migrating topics from classic to diskless.
The initialization transfers log offsets and producer state.
@giuseppelillo giuseppelillo force-pushed the giuseppelillo/init-diskless-log branch from 79bea61 to 1329854 Compare January 19, 2026 14:29
CREATE DOMAIN leader_epoch_t AS INT NOT NULL
CHECK (VALUE >= 0);

ALTER TABLE logs ADD COLUMN diskless_start_offset offset_t DEFAULT 0;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we will have 0 as default here, how we will further distinguish the topics that are in process of migration and those that are created as diskless for the start? Will we use this offset for routing the fetch logic to tiered vs diskless? If so then will 0 mean that there is no tiered data?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we will have 0 as default here, how we will further distinguish the topics that are in process of migration and those that are created as diskless for the start?

Once the entry in the logs table is present, it means the migration was successful. All the necessary information transfer are made within the same transaction (producer state, high watermark, ...)

Will we use this offset for routing the fetch logic to tiered vs diskless? If so then will 0 mean that there is no tiered data?

Yes exactly, diskless_start_offset represents the boundary B0 of the design doc. So diskless_start_offset == 0 means no tiered data is present. This field will then be updated after the merge to TS has been performed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But will it be 0 after the migration initialization for tiered partition right away since it's the default?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I get it now. I thought for a moment that this default will be set for all the diskless + hybrid/migrated topics. But do I now understand correctly that it will be set only for the partitions that were created as diskless from the very beginning?

Comment on lines +85 to +91
UPDATE logs
SET log_start_offset = l_request.log_start_offset,
high_watermark = l_request.diskless_start_offset,
diskless_start_offset = l_request.diskless_start_offset,
leader_epoch_at_init = l_request.leader_epoch
WHERE topic_id = l_request.topic_id
AND partition = l_request.partition;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we actually do updates here? I suspect that if we use diskless_log_start for routing decisions then it could be already observed by one of the nodes so the routing decision might be incorrect in this case

long logStartOffset = 0;
long highWatermark = 0;
long byteSize = 0;
Long disklessStartOffset = null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But for PostgresControlPlane it is 0, right? Should not they be the same?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, i'll fix this

INTO l_existing_log
FROM logs
WHERE topic_id = l_request.topic_id
AND partition = l_request.partition;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to lock here with FOR UPDATE?

FOR l_request IN
SELECT *
FROM unnest(arg_requests)
LOOP
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also check here that
l_request.log_start_offset <= l_request.diskless_start_offset
?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants