Commit Graph

20 Commits

Author SHA1 Message Date
Michael Litvak
e7dbccd59e cdc: use chunked_vector instead of vector for stream ids
use utils::chunked_vector instead of std::vector to store cdc stream
sets for tablets.

a cdc stream set usually represents all streams for a specific table and
timestamp, and has a stream id per each tablet of the table. each stream
id is represented by 16 bytes. thus the vector could require quite large
contiguous allocations for a table that has many tablets. change it to
chunked_vector to avoid large contiguous allocations.

Fixes scylladb/scylladb#26791

Closes scylladb/scylladb#26792
2025-10-31 13:02:34 +01:00
Michael Litvak
8743422241 cdc: improve cdc metadata loading
when loading CDC streams metadata for tablets from the tables, read only
new entries from the history table instead of reading all entries. This
improves the CDC metadata reloading, making it more efficient and
predictable.

the CDC metadata is loaded as part of group0 reload whenever the
internal CDC tables are modified. on tablet split / merge, we create a
new CDC timestamp and streams by writing them to the cdc_streams_history
table by group0 operation, and when it's applied we reload the in-memory
CDC streams map by reading from the tables and constructing the updated map.

Previously, on every update, we would read the entire
cdc_streams_history entries for the changed table, constructing all its
streams and creating a new map from scratch.

We improve this now by reading only new entries from cdc_streams_history
and append them to the existing map. we can do this because we only
append new entries to cdc_streams_history with higher timestamp than all
previous entries.

This makes this reloading more efficient and predictable, because
previously we would read a number of entries that depends on the number
of tablets splits and merges, which increases over time and is
unbounded, whereas now we read only a single stream set on each update.

Fixes scylladb/scylladb#26732
2025-10-28 08:54:09 +01:00
Michael Litvak
67410cac4d cdc: generate_stream_diff helper function
This helper functions receives two sets of streams and constructs their
difference - closed and opened streams.
2025-09-17 14:47:12 +02:00
Michael Litvak
5f6bb0af9d cdc: choose stream in tablets enabled keyspaces
When choosing a CDC stream to generate CDC log writes to, if the
keyspace uses tablets, we need to choose a stream according to the
relevant metadata which is specific to tablets-enabled keyspaces.

We define the method get_tablet_stream that given a table, write
timestamp, and token, returns the stream that the log entry should
be written to.

The method works by looking up the stream metadata of the table, then
finding the relevant stream set by timestamp, and finally finding
the stream that covers the token range that contains the token.
2025-09-17 14:47:12 +02:00
Michael Litvak
28cdd81ef0 cdc: rename get_stream to get_vnode_stream
the get_stream method is relevant only for vnode-based keyspaces. next
we will introduce a new method to get a stream in a tablets-based
keyspace. prepare for this by renaming get_stream to get_vnode_stream.
2025-09-17 14:47:12 +02:00
Michael Litvak
9ec4b6ccb1 cdc: load tablet streams metadata from tables
Read the CDC stream metadata from the internal system tables, and store
it in the cdc metadata data structures.

The metadata is stored in the tables as diffs which is more storage
efficient, but when in-memory we store it as full stream sets for each
timestamp. This is more useful because we need to be able to find a
stream given timestamp and token.
2025-09-17 14:47:12 +02:00
Michael Litvak
4f5550d7f2 cdc: fix handling of new generation during raft upgrade
During raft upgrade, a node may gossip about a new CDC generation that
was propagated through raft. The node that receives the generation by
gossip may have not applied the raft update yet, and it will not find
the generation in the system tables. We should consider this error
non-fatal and retry to read until it succeeds or becomes obsolete.

Another issue is when we fail with a "fatal" exception and not retrying
to read, the cdc metadata is left in an inconsistent state that causes
further attempts to insert this CDC generation to fail.

What happens is we complete preparing the new generation by calling `prepare`,
we insert an empty entry for the generation's timestamp, and then we fail. The
next time we try to insert the generation, we skip inserting it because we see
that it already has an entry in the metadata and we determine that
there's nothing to do. But this is wrong, because the entry is empty,
and we should continue to insert the generation.

To fix it, we change `prepare` to return `true` when the entry already
exists but it's empty, indicating we should continue to insert the
generation.

Fixes scylladb/scylladb#21227

Closes scylladb/scylladb#22093
2025-01-28 18:05:32 +01:00
Avi Kivity
f3eade2f62 treewide: relicense to ScyllaDB-Source-Available-1.0
Drop the AGPL license in favor of a source-available license.
See the blog post [1] for details.

[1] https://www.scylladb.com/2024/12/18/why-were-moving-to-a-source-available-license/
2024-12-18 17:45:13 +02:00
Kefu Chai
3e84d43f93 treewide: use seastar::format() or fmt::format() explicitly
before this change, we rely on `using namespace seastar` to use
`seastar::format()` without qualifying the `format()` with its
namespace. this works fine until we changed the parameter type
of format string `seastar::format()` from `const char*` to
`fmt::format_string<...>`. this change practically invited
`seastar::format()` to the club of `std::format()` and `fmt::format()`,
where all members accept a templated parameter as its `fmt`
parameter. and `seastar::format()` is not the best candidate anymore.
despite that argument-dependent lookup (ADT for short) favors the
function which is in the same namespace as its parameter, but
`using namespace` makes `seastar::format()` more competitive,
so both `std::format()` and `seastar::format()` are considered
as the condidates.

that is what is happening scylladb in quite a few caller sites of
`format()`, hence ADT is not able to tell which function the winner
in the name lookup:

```
/__w/scylladb/scylladb/mutation/mutation_fragment_stream_validator.cc:265:12: error: call to 'format' is ambiguous
  265 |     return format("{} ({}.{} {})", _name_view, s.ks_name(), s.cf_name(), s.id());
      |            ^~~~~~
/usr/bin/../lib/gcc/x86_64-redhat-linux/14/../../../../include/c++/14/format:4290:5: note: candidate function [with _Args = <const std::basic_string_view<char> &, const seastar::basic_sstring<char, unsigned int, 15> &, const seastar::basic_sstring<char, unsigned int, 15> &, const utils::tagged_uuid<table_id_tag> &>]
 4290 |     format(format_string<_Args...> __fmt, _Args&&... __args)
      |     ^
/__w/scylladb/scylladb/seastar/include/seastar/core/print.hh:143:1: note: candidate function [with A = <const std::basic_string_view<char> &, const seastar::basic_sstring<char, unsigned int, 15> &, const seastar::basic_sstring<char, unsigned int, 15> &, const utils::tagged_uuid<table_id_tag> &>]
  143 | format(fmt::format_string<A...> fmt, A&&... a) {
      | ^
```

in this change, we

change all `format()` to either `fmt::format()` or `seastar::format()`
with following rules:
- if the caller expects an `sstring` or `std::string_view`, change to
  `seastar::format()`
- if the caller expects an `std::string`, change to `fmt::format()`.
  because, `sstring::operator std::basic_string` would incur a deep
  copy.

we will need another change to enable scylladb to compile with the
latest seastar. namely, to pass the format string as a templated
parameter down to helper functions which format their parameters.
to miminize the scope of this change, let's include that change when
bumping up the seastar submodule. as that change will depend on
the seastar change.

Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>
2024-09-11 23:21:40 +03:00
Kefu Chai
1a4740ddc0 cdc: do not include unused headers
these unused includes were identified by clangd. see
https://clangd.llvm.org/guides/include-cleaner#unused-include-warning
for more details on the "Unused include" warning.

Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>
2024-06-21 14:29:48 +08:00
Patryk Jędrzejczak
0470b721c2 cdc: generation: allow increasing generation_leeway through error injection
The increased `generation_leeway` is used in the next patch to
write a test. Since it's no longer a constant, we create a new
getter for it.
2024-02-12 10:14:00 +01:00
Patryk Jędrzejczak
330a37b5c9 cdc: metadata: allow sending writes to the previous generations
Before this patch, writes to the previous CDC generations would
always be rejected. After this patch, they will be accepted if
the write's timestamp is greater than `now - generation_leeway`.

This change was proposed around 3 years ago. The motivation was
to improve user experience. If a client generates timestamps by
itself and its clock is desynchronized with the clock of the node
the client is connected to, there could be a period during
generation switching when writes fail. We didn't consider this
problem critical because the client could simply retry a failed
write with a higher timestamp. Eventually, it would succeed. This
approach is safe because these failed writes cannot have any side
effects. However, it can be inconvenient. Writing to previous
generations was proposed to improve it.

The idea was rejected 3 years ago. Recently, it turned out that
there is a case when the client cannot retry a write with the
increased timestamp. It happens when a table uses CDC and LWT,
which makes timestamps permanent. Once Paxos commits an entry with
a given timestamp, Scylla will keep trying to apply that entry
until it succeeds, with the same timestamp. Applying the entry
involves writing to the CDC log table. If it fails, we get stuck.
It's a major bug with an unknown perfect solution.

Allowing writes to previous generations for `generation_leeway` is
a probabilistic fix that should solve the problem in practice.

Note that allowing writes only to the previous generation might
not be enough. With the Raft-based topology, it is possible to
add multiple nodes concurrently. Moreover, tablets make streaming
instant, which allows the topology coordinator to add multiple nodes
very quickly. So, creating generations with almost identical
timestamps is possible. Then, we could encounter the same bug but,
for example, for a generation before the previous generation.
2024-02-12 10:14:00 +01:00
Michael Huang
62a8a31be7 cdc: use chunked_vector for topology_description entries
Lists can grow very big. Let's use a chunked vector to prevent large contiguous
allocations.
Fixes: #15302.

Closes scylladb/scylladb#15428
2023-09-18 23:17:01 +03:00
Avi Kivity
fcb8d040e8 treewide: use Software Package Data Exchange (SPDX) license identifiers
Instead of lengthy blurbs, switch to single-line, machine-readable
standardized (https://spdx.dev) license identifiers. The Linux kernel
switched long ago, so there is strong precedent.

Three cases are handled: AGPL-only, Apache-only, and dual licensed.
For the latter case, I chose (AGPL-3.0-or-later and Apache-2.0),
reasoning that our changes are extensive enough to apply our license.

The changes we applied mechanically with a script, except to
licenses/README.md.

Closes #9937
2022-01-18 12:15:18 +01:00
Avi Kivity
a55b434a2b treewide: extent copyright statements to present day 2021-06-06 19:18:49 +03:00
Piotr Jastrzebski
649f254863 cdc: Limit size of topology description
Currently, whole topology description for CDC is stored in a single row.
This means that for a large cluster of strong machines (say 100 nodes 64
cpus each), the size of the topology description can reach 32MB.

This causes multiple problems. First of all, there's a hard limit on
mutation size that can be written to Scylla. It's related to commit log
block size which is 16MB by default. Mutations bigger than that can't be
saved. Moreover, such big partitions/rows cause reactor stalls and
negatively influence latency of other requests.

This patch limits the size of topology description to about 4MB. This is
done by reducing the number of CDC streams per vnode and can lead to CDC
data not being fully colocated with Base Table data on shards. It can
impact performance and consistency of data.

This is just a quick fix to make it easily backportable. A full solution
to the problem is under development.

For more details see #7961, #7993 and #7985.

Signed-off-by: Piotr Jastrzebski <piotr@scylladb.com>
2021-02-17 13:24:40 +01:00
Calle Wilund
05851578d4 alternator::streams: Report streams as not ready until CDC stream id:s are available
Refs #6864

When booting a clean scylla, CDC stream ID:s will not be availble until
a n*ring delay time period has passed. Before this, writing to a CDC
enabled table will fail hard.
For alternator (and its tests), we can report the stream(s) for tables as not yet
available (ENABLING) until such time as id:s are
computed.

v2:
* Keep storage service ref in executor
2020-08-03 20:34:15 +03:00
Piotr Jastrzebski
f0f6e220ea cdc: stop using partitioners
CDC can get all it needs from a config and does not need
partitioner.

For base table specific operations CDC is using partitioner
from that table (obtained with schema::get_partitioner).

Signed-off-by: Piotr Jastrzebski <piotr@scylladb.com>
2020-02-17 10:59:15 +01:00
Piotr Jastrzebski
50cfe81331 murmur3: move sharding logic to token and i_partitioner
Since token representation is fixed now, all the partitioners
will share the sharding logic. It makes sense now to keep
the logic in common super class and separate header that's
included only in i_partitioner.cc.

shard_of and token_for_next_shard are now implemented in
i_partitioner. They would be non-virtual but we have to
keep them virtual because one test is overriding them
to enforce some specific sharding.

Signed-off-by: Piotr Jastrzebski <piotr@scylladb.com>
2020-02-05 09:31:32 +01:00
Kamil Braun
834c2ca997 cdc: add cdc::metadata class
The class stores a queue of CDC generations to be used for choosing
streams when writing to the CDC log.

This data structure will be updated on some gossip events (when a new node
joins the cluster and proposes a new generation of CDC streams).
2020-01-30 11:10:08 +01:00