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

raft topology: replace publish_cdc_generation with a bg fiber #15281

Merged
merged 8 commits into from
Sep 8, 2023

Conversation

patjed41
Copy link
Contributor

@patjed41 patjed41 commented Sep 5, 2023

Currently, the topology coordinator has the topology::transition_state::publish_cdc_generation state responsible for publishing the already created CDC generations to the user-facing description tables. This process cannot fail as it would cause some CDC updates to be missed. On the other hand, we would like to abort the publish_cdc_generation state when bootstrap aborts. Of course, we could also wait until handling this state finishes, even in the case of the bootstrap abort, but that would be inefficient. We don't want to unnecessarily block topology operations by publishing CDC generations.

The solution proposed by this PR is to remove the publish_cdc_generation state completely and introduce a new background fiber of the topology coordinator -- cdc_generation_publisher -- that continually publishes committed CDC generations.

Apart from introducing the CDC generation publisher, we add test_cdc_generation_publishing.py that verifies its correctness and we adapt other CDC tests to the new changes.

Fixes #15194

@patjed41
Copy link
Contributor Author

patjed41 commented Sep 5, 2023

In #15194, the proposed solution was to modify system_distributed_keyspace::create_cdc_desc and publish all CDC generations atomically at once. I don't know if it is worth implementing. For now, I've implemented an easier iterative solution. Although it seems less efficient, the number of unpublished CDC generations shouldn't exceed one in vast majority of cases, so in practice there should be almost no difference. Changing the current solution to the proposed one requires only a few small modifications of the current PR (apart from system_distributed_keyspace::create_cdc_desc) so I can easily change it if we decide to go for it.

@gleb-cloudius
Copy link
Contributor

In #15194, the proposed solution was to modify system_distributed_keyspace::create_cdc_desc and publish all CDC generations atomically at once. I don't know if it is worth implementing. For now, I've implemented an easier iterative solution. Although it seems less efficient, the number of unpublished CDC generations shouldn't exceed one in vast majority of cases, so in practice there should be almost no difference. Changing the current solution to the proposed one requires only a few small modifications of the current PR (apart from system_distributed_keyspace::create_cdc_desc) so I can easily change it if we decide to go for it.

@kbr-scylla asked for atomicity since otherwise a driver may see later generations timestamps earlier and miss some updates. By "iterative" solution do you mean that you publish CDC generations one by one and do not start later one until earlier is fully published?

@patjed41
Copy link
Contributor Author

patjed41 commented Sep 5, 2023

By "iterative" solution do you mean that you publish CDC generations one by one and do not start later one until earlier is fully published?

The generations are published in non-decreasing order of timestamps.

}
} catch (raft::request_aborted&) {
slogger.debug("raft topology: CDC generation publisher fiber aborted");
} catch (term_changed_error&) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I do not think we should ever get it here. Happens when we try to retake a node.

Copy link
Contributor

Choose a reason for hiding this comment

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

No, I am taking this back. start_operation() may throw it. _as should be aborted at this point, but we do not know what happens first us checking _as or getting the exception.

@@ -410,8 +410,6 @@ future<> storage_service::topology_state_load() {
[[fallthrough]];
case topology::transition_state::commit_cdc_generation:
[[fallthrough]];
case topology::transition_state::publish_cdc_generation:
Copy link
Contributor

Choose a reason for hiding this comment

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

update_topology_state does not handle group0_concurrent_modification. It only logs and re-throws.

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 somehow missed this throw;. I fixed it.

@scylladb-promoter
Copy link
Contributor

@patjed41
Copy link
Contributor Author

patjed41 commented Sep 6, 2023

v2:

  • added catch (group0_concurrent_modification&) {} to cdc_generation_publisher_fiber
  • increased the timeout of waiting for the message in the cdc_generation_publisher_fiber error injection from 1 minute to 2 minutes

@patjed41
Copy link
Contributor Author

patjed41 commented Sep 6, 2023

CI state FAILURE - https://jenkins.scylladb.com/job/scylla-master/job/scylla-ci/3473/

Waiting for the message in the cdc_generation_publisher_fiber error injection added by me timed out in debug mode. Logs suggest to me that the 1 minute timout is almost enough. For now, I increased the timeout from 1 minute to 2 minutes and I think it should be enough to make test_multiple_unpublished_cdc_generations pass every time, but we will see. Another option is to skip this test in debug mode.

@scylladb-promoter
Copy link
Contributor

@@ -137,6 +137,9 @@ struct topology {
// It's used as partition key in CDC_GENERATIONS_V3 table.
std::optional<utils::UUID> new_cdc_generation_data_uuid;

// The IDs of the commited yet unpublished CDC generations sorted by timestamps.
Copy link
Contributor

Choose a reason for hiding this comment

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

We rely on the set type ordering the items, and then tuples being ordered lexicographically, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, exactly.

service/storage_service.cc Outdated Show resolved Hide resolved
@@ -1227,6 +1227,12 @@ class topology_coordinator {
slogger.trace("raft topology: start CDC generation publisher fiber");

while (!_as.abort_requested()) {
co_await utils::get_local_injector().inject_with_handler("cdc_generation_publisher_fiber", [] (auto& handler) -> future<> {
slogger.info("raft toplogy: CDC generation publisher fiber sleeps after injection");
co_await handler.wait_for_message(std::chrono::steady_clock::now() + std::chrono::minutes{2});
Copy link
Contributor

Choose a reason for hiding this comment

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

Very optimistic to assume that you'll manage to bootstrap 3 nodes in 2 minutes on our great CI machines with all these concurrent tests :) And let's not even talk about ARM.

Let's increase the timeout to 5 minutes. And... let's mark the two tests dev-mode only.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

await handler.message()
while len(gen_timestamps) < 4:
# We prefer to detect CDC generation publications one-by-one, because it increases our chances of catching
# potential bugs like incorrect order of publications. Therefore, we use very short period - 0.01 s.
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice.

query = SimpleStatement(
"select time from system_distributed.cdc_generation_timestamps where key = 'timestamps'",
consistency_level=ConsistencyLevel.ONE)
while len(list(cql.execute(query))) == 0:
Copy link
Contributor

Choose a reason for hiding this comment

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

This should have a time limit.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

@patjed41
Copy link
Contributor Author

patjed41 commented Sep 7, 2023

v3:

  • fixed the name of set_unpublisheded_cdc_generations
  • increased the timeout of waiting for the message in the cdc_generation_publisher_fiber error injection to 5 minutes
  • made test_cdc_generation_publishing.py dev-only
  • added timeout to wait_for_first_cdc_generation

@scylladb-promoter
Copy link
Contributor

In the following commits, we replace the
topology::transition_state::publish_cdc_generation state with
a background fiber that continually publishes committed CDC
generations. To make these generations accessible to the
topology coordinator, we store them in the new column of
system.topology -- unpublished_cdc_generations.
We add committed CDC generations to unpublished_cdc_generations
so that we can load them to topology and properly handle them
in the following commits.
We extend service::topology with the list of unpublished CDC
generations and load its contents from system.topology. This step
is the last one in making unpublished CDC generations accessible
to the topology coordinator.

Note that when we load unpublished_cdc_generations, we don't
perform any sanity checks contrary to current_cdc_generation_uuid.
Every unpublished CDC generation was a current generation once,
and we checked it at that moment.
Currently, the topology coordinator has the
topology::transition_state::publish_cdc_generation state
responsible for publishing the already created CDC generations
to the user-facing description tables. This process cannot fail
as it would cause some CDC updates to be missed. On the other
hand, we would like to abort the publish_cdc_generation state when
bootstrap aborts. Of course, we could also wait until handling this
state finishes, even in the case of the bootstrap abort, but that
would be inefficient. We don't want to unnecessarily block topology
operations by publishing CDC generations.

The solution is to remove the publish_cdc_generation state
completely and introduce a new background fiber of the topology
coordinator -- cdc_generation_publisher -- that continually
publishes committed CDC generations.

The implementation of the CDC generation publisher is very similar
to the main fiber of the topology coordinator. One noticeable
difference is that we don't catch raft::commit_status_unknown,
which is handled raft_group0_client::add_entry.

Note that this modification changes the Raft-based topology a bit.
Previously, the publish_cdc_generation state had to end before
entering the next state -- write_both_read_old. Now, committed
CDC generations can theoretically be published at any time.
Although it is correct because the following states don't depend on
publish_cdc_generation, it can cause problems in tests. For example,
we can't assume now that a CDC generation is published just because
the bootstrap operation has finished.
We update documentation after replacing the
topology::transition_state::publish_cdc_generation state with
the CDC generation publisher fiber.
We add two test cases that test the new CDC generation publisher
to detect potential bugs like incorrect order of publications or
not publishing some generations at all.

The purpose of the second test case --
test_multiple_unpublished_cdc_generations -- is to enforce and test
a scenario when there are multiple unpublished CDC generations at
the same time. We expect that this is a rare case. The main fiber
of the topology coordinator would have to make much more progress
(like finishing two bootstraps) than the CDC generation publisher
fiber. Since multiple unpublished CDC generations might never
appear in other tests but could be handled incorrectly, having
such a test is valuable.
The part of test_topology_ops that tests the
cdc_streams_check_and_repair request could (at least in theory)
fail on
`assert(len(gen_timestamps) + 1 == len(new_gen_timestamps))`
after introducing the CDC generation publisher because we can
no longer assume that all previously committed CDC generations
have been published before sending the request.

To prevent flakiness, we move this part of the test to
test_cdc_generations_are_published. This test allows for ensuring
that all previous CDC generations have been published.
Additionally, checking cdc_streams_check_and_repair there is
simpler and arguably fits the test better.
After introducing the CDC generation publisher,
test_cdc_log_entries_use_cdc_streams could (at least in theory)
fail by accessing system_distributed.cdc_streams_descriptions_v2
before the first CDC generation has been published.

To avoid flakiness, we simply wait until the first CDC generation
is published in a new function -- wait_for_first_cdc_generation.
@patjed41
Copy link
Contributor Author

patjed41 commented Sep 8, 2023

I only rebased on newer master. I realized #15319 conflicts with this PR and I need to rebase it sooner or later, but I must firstly match their versions of master.

@scylladb-promoter
Copy link
Contributor

@scylladb-promoter scylladb-promoter merged commit 26d9a82 into scylladb:master Sep 8, 2023
4 checks passed
kbr-scylla added a commit that referenced this pull request Sep 18, 2023
…ions table' from Patryk Jędrzejczak

We add garbage collection for the `CDC_GENERATIONS_V3` table to prevent
it from endlessly growing. This mechanism is especially needed because
we send the entire contents of `CDC_GENERATIONS_V3` as a part of the
group 0 snapshot.

The solution is to keep a clean-up candidate, which is one of the
already published CDC generations. The CDC generation publisher
introduced in #15281 continually uses this candidate to remove all
generations with timestamps not exceeding the candidate's and sets a new
candidate when needed.

We also add `test_cdc_generation_clearing.py` that verifies this new
mechanism.

Fixes #15323

Closes #15413

* github.com:scylladb/scylladb:
  test: add test_cdc_generation_clearing
  raft topology: remove obsolete CDC generations
  raft topology: set CDC generation clean-up candidate
  topology_coordinator: refactor publish_oldest_cdc_generation
  system_keyspace: introduce decode_cdc_generation_id
  system_keyspace: add cleanup_candidate to CDC_GENERATIONS_V3
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants