Commit Graph

48763 Commits

Author SHA1 Message Date
Botond Dénes
dbdd5cf10b Merge '[Backport 2025.3] db: batchlog_manager: update _last_replay only if all batches were re…' from Scylladb[bot]
…played

Currently, if flushing hints falls within the repair cache timeout, then the flush_time is set to batchlog_manager::_last_replay. _last_replay is updated on each replay, even if some batches weren't replayed. Due to that, we risk the data resurrection.

Update _last_replay only if all batches were replayed.

Fixes: https://github.com/scylladb/scylladb/issues/24415.

Needs backport to all live versions.

- (cherry picked from commit 4d0de1126f)

- (cherry picked from commit e3dcb7e827)

Parent PR: #26793

Closes scylladb/scylladb#27092

* github.com:scylladb/scylladb:
  test: extend test_batchlog_replay_failure_during_repair
  db: batchlog_manager: update _last_replay only if all batches were replayed
2026-01-30 16:20:47 +02:00
Botond Dénes
3d9f38ccbb db/row_cache: make_nonpopulating_reader(): pass cache tracker to snapshot
The API contract in partition_version.hh states that when dealing with
evictable entries, a real cache tracker pointer has to be passed to all
methods that ask for it. The nonpopulating reader violates this, passing
a nullptr to the snapshot. This was observed to cause a crash when a
concurrent cache read accessed the snapshot with the null tracker.

A reproducer is included which fails before and passes after the fix.

Fixes: #26847

Closes scylladb/scylladb#28163

(cherry picked from commit a53f989d2f)

Closes scylladb/scylladb#28278
2026-01-30 16:15:26 +02:00
Patryk Jędrzejczak
43ee6ba035 test: test_gossiper_orphan_remover: get host ID of the bootstrapping node before it crashes
The test is currently flaky. It tries to get the host ID of the bootstrapping
node via the REST API after the node crashes. This can obviously fail. The
test usually doesn't fail, though, as it relies on the host ID being saved
in `ScyllaServer._host_id` at this point by `ScyllaServer.try_get_host_id()`
repeatedly called in `ScyllaServer.start()`. However, with a very fast crash
and unlucky timings, no such call may succeed.

We deflake the test by getting the host ID before the crash. Note that at this
point, the bootstrapping node must be serving the REST API requests because
`await log.wait_for("finished do_send_ack2_msg")` above guarantees that the
node has started the gossip shadow round, which happens after starting the REST
API.

Fixes #28385

Closes scylladb/scylladb#28388

(cherry picked from commit a2c1569e04)

Closes scylladb/scylladb#28415
2026-01-29 11:31:36 +01:00
Yaron Kaikov
02994a0a3e .github/workflows/backport-pr-fixes-validation: support Atlassian URL format in backport PR fixes validation
Add support for matching full Atlassian JIRA URLs in the format
https://scylladb.atlassian.net/browse/SCYLLADB-400 in addition to
the bare JIRA key format (SCYLLADB-400).

This makes the validation more flexible by accepting both formats
that developers commonly use when referencing JIRA issues.

Fixes: https://github.com/scylladb/scylladb/issues/28373

Closes scylladb/scylladb#28374

(cherry picked from commit 3f10f44232)

Closes scylladb/scylladb#28392
2026-01-27 16:06:44 +02:00
Gleb Natapov
826c323c04 topology coordinator: complete pending operation for a replaced node
A replaced node may have pending operation on it. The replace operation
will move the node into the 'left' state and the request will never be
completed. More over the code does not expect left node to have a
request. It will try to process the request and will crash because the
node for the request will not be found.

The patch checks is the replaced node has peening request and completes
it with failure. It also changes topology loading code to skip requests
for nodes that are in a left state. This is not strictly needed, but
makes the code more robust.

Fixes #27990

Closes scylladb/scylladb#28009

(cherry picked from commit bee5f63cb6)

Closes scylladb/scylladb#28177
2026-01-26 11:43:36 +01:00
Patryk Jędrzejczak
e1bd21db7e test: test_raft_recovery_during_join: get host ID of the bootstrapping node before it crashes
The test is currently flaky. It tries to get the host ID of the bootstrapping
node via the REST API after the node crashes. This can obviously fail. The
test usually doesn't fail, though, as it relies on the host ID being saved
in `ScyllaServer._host_id` at this point by `ScyllaServer.try_get_host_id()`
repeatedly called in `ScyllaServer.start()`. However, with a very fast crash
and unlucky timings, no such call may succeed.

We deflake the test by getting the host ID before the crash. Note that at this
point, the bootstrapping node must be serving the REST API requests because
`await coordinator_log.wait_for("delay_node_bootstrap: waiting for message")`
above guarantees that the node has submitted the join topology request, which
happens after starting the REST API.

Fixes #28227

Closes scylladb/scylladb#28233

(cherry picked from commit e503340efc)

Closes scylladb/scylladb#28309
2026-01-22 13:18:56 +01:00
Patryk Jędrzejczak
5577ad4881 test: test_zero_token_nodes_multidc: properly handle reads with CL=LOCAL_ONE
The test is currently flaky. It incorrectly assumes that a read with
CL=LOCAL_ONE will see the data inserted by a preceding write with
CL=LOCAL_ONE in the same datacenter with RF=2.

The same issue has already been fixed for CL=ONE in
21edec1ace. The difference is that
for CL=LOCAL_ONE, only dc1 is problematic, as dc2 has RF=1.

We fix the issue for CL=LOCAL_ONE by skipping the check for dc1.

Fixes #28253

The fix addresses CI flakiness and only changes the test, so it
should be backported.

Closes scylladb/scylladb#28274

(cherry picked from commit 1f0f694c9e)

Closes scylladb/scylladb#28303
2026-01-22 11:04:46 +01:00
Aleksandra Martyniuk
cd5508a690 test: extend test_batchlog_replay_failure_during_repair
Modify test_batchlog_replay_failure_during_repair to also check
that there isn't data resurrection if flushing hints falls within
the repair cache timeout.

(cherry picked from commit e3dcb7e827)
2026-01-22 10:39:18 +01:00
Aleksandra Martyniuk
c3a9415e0c db: batchlog_manager: update _last_replay only if all batches were replayed
Currently, if flushing hints falls within the repair cache timeout,
then the flush_time is set to batchlog_manager::_last_replay.
_last_replay is updated on each replay, even if some batches weren't
replayed. Due to that, we risk the data resurrection.

Update _last_replay only if all batches were replayed.

Fixes: https://github.com/scylladb/scylladb/issues/24415.
(cherry picked from commit 4d0de1126f)
2026-01-22 10:39:00 +01:00
Tomasz Grabiec
afa4d60ac1 Fix lambda-coroutine fiasco in hint_endpoint_manager.cc
Found by copilot.

No issue was observed yet.

Fixes #27520

Closes scylladb/scylladb#27477

(cherry picked from commit 7bc59e93b2)

Closes scylladb/scylladb#27730
2026-01-21 14:15:36 +02:00
Anna Stuchlik
6dd7752e02 doc: fix the default compaction strategy for Materialized Views
Fixes https://github.com/scylladb/scylladb/issues/24483

Closes scylladb/scylladb#27725

(cherry picked from commit 84e9b94503)

Closes scylladb/scylladb#28284
2026-01-21 06:45:36 +02:00
Botond Dénes
cb6b20e197 reader_concurrency_semaphore: improve handling of base resources
reader_permit::release_base_resources() is a soft evict for the permit:
it releases the resources aquired during admission. This is used in
cases where a single process owns multiple permits, creating a risk for
deadlock, like it is the case for repair. In this case,
release_base_resources() acts as a manual eviction mechanism to prevent
permits blockings each other from admission.

Recently we found a bad interaction between release_base_resources() and
permit eviction. Repair uses both mechanism: it marks its permits as
inactive and later it also uses release_base_resources(). This partice
might be worth reconsidering, but the fact remains that there is a bug
in the reader permit which causes the base resources to be released
twice when release_base_resources() is called on an already evicted
permit. This is incorrect and is fixed in this patch.

Improve release_base_resources():
* make _base_resources const
* move signal call into the if (_base_resources_consumed()) { }
* use reader_permit::impl::signal() instead of
  reader_concurrency_semaphore::signal()
* all places where base resources are released now call
  release_base_resources()

A reproducer unit test is added, which fails before and passes after the
fix.

Fixes: #28083

Closes scylladb/scylladb#28155

(cherry picked from commit b7bc48e7b7)

Closes scylladb/scylladb#28242
2026-01-21 06:45:13 +02:00
Aleksandra Martyniuk
94607c36e0 service: node_ops: remove coroutine::lambda wrappers
In storage_service::raft_topology_cmd_handler we pass a lambda
wrapped in coroutine::lambda to a function that creates streaming_task_impl.
The lambda is kept in streaming_task_impl that invokes it in its run
method.

The lambda captures may be destroyed before the lambda is called, leading
to use after free.

Do not wrap a lambda passed to streaming_task_impl into coroutine::lambda.
Use this auto dissociate the lambda lifetime from the calling statement.

Fixes: https://github.com/scylladb/scylladb/issues/28200.

Closes scylladb/scylladb#28201

(cherry picked from commit 65cba0c3e7)

Closes scylladb/scylladb#28241
2026-01-21 06:44:45 +02:00
Ernest Zaslavsky
161b66759c aws_error: fix nested exception handling
The loop that unwraps nested exception, rethrows nested exception and saves pointer to the temporary std::exception& inner on stack, then continues. This pointer is, thus, pointing to a released temporary

Closes scylladb/scylladb#28143

(cherry picked from commit 829bd9b598)

Closes scylladb/scylladb#28239
2026-01-20 11:19:06 +01:00
Asias He
bc8bbf2caa repair: Allow min max range to be updated for repair history
It is observed that:

repair - repair[667d4a59-63fb-4ca6-8feb-98da49946d8b]: Failed to update
system.repair_history table of node d27de212-6f32-4649ad76-a9ef1165fdcb:
seastar::rpc::remote_verb_error
(repair[667d4a59-63fb-4ca6-8feb-98da49946d8b]: range (minimum
token,maximum token) is not in the format of (start, end])

This is because repair checks the end of the range to be repaired needs
to be inclusive. When small_table_optimization is enabled for regular
repair, a (minimum token,maximum token) will be used.

To fix, we can relax the check of (start, end] for the min max range.

Fixes #27220

Backport to all active branches.

(cherry picked from commit e97a504)
Parent PR: #27357

Closes scylladb/scylladb#27460
2026-01-19 09:40:55 +02:00
Nikos Dragazis
f89af142da test: database_test: Fix serialization of partition key
The `make_key` lambda erroneously allocates a fixed 8-byte buffer
(`sizeof(s.size())`) for variable-length strings, potentially causing
uninitialized bytes to be included. If such bytes exist and they are
not valid UTF-8 characters, deserialization fails:

```
ERROR 2026-01-16 08:18:26,062 [shard 0:main] testlog - snapshot_list_contains_dropped_tables: cql env callback failed, error: exceptions::invalid_request_exception (Exception while binding column p1: marshaling error: Validation failed - non-UTF8 character in a UTF8 string, at byte offset 7)
```

Fixes #28195.

Signed-off-by: Nikos Dragazis <nikolaos.dragazis@scylladb.com>

Closes scylladb/scylladb#28197

(cherry picked from commit 8aca7b0eb9)

Closes scylladb/scylladb#28208
2026-01-19 09:40:25 +02:00
Tomasz Grabiec
6b88add507 Merge '[Backport 2025.3] topology_coordinator: Add barrier to cleanup_target' from Scylladb[bot]
Consider the following scenario:
1. A table has RF=3 and writes use CL=QUORUM
2. One node is down
3. There is a pending tablet migration from the unavailable node
   that is reverted

During the revert, there can be a time window where the pending replica
being cleaned up still accepts writes. This leads to write failures,
as only two nodes (out of four) are able to acknowledge writes.

This patch fixes the issue by adding a barrier to the cleanup_target
tablet transition state, ensuring that the coordinator switches back to
the previous replica set before cleanup is triggered.

Fixes https://github.com/scylladb/scylladb/issues/26512

It's a pre existing issue. Backport is required to all recent 2025.x versions.

- (cherry picked from commit 669286b1d6)

- (cherry picked from commit 67f1c6d36c)

- (cherry picked from commit 6163fedd2e)

Parent PR: #27413

Closes scylladb/scylladb#27427

* github.com:scylladb/scylladb:
  topology_coordinator: Fix the indentation for the cleanup_target case
  topology_coordinator: Add barrier to cleanup_target
  test_node_failure_during_tablet_migration: Increase RF from 2 to 3
2026-01-16 16:31:52 +01:00
Calle Wilund
6a7c9cd750 db::commitlog: Fix sanity check error on race between segment flushing and oversized alloc
Fixes #27992

When doing a commit log oversized allocation, we lock out all other writers by grabbing
the _request_controller semaphore fully (max capacity).
We thereafter assert that the semaphore is in fact zero. However, due to how things work
with the bookkeep here, the semaphore can in fact become negative (some paths will not
actually wait for the semaphore, because this could deadlock).

Thus, if, after we grab the semaphore and execution actually returns to us (task schedule),
new_buffer via segment::allocate is called (due to a non-fully-full segment), we might
in fact grab the segment overhead from zero, resulting in a negative semaphore.

The same problem applies later when we try to sanity check the return of our permits.

Fix is trivial, just accept less-than-zero values, and take same possible ltz-value
into account in exit check (returning units)

Added whitebox (special callback interface for sync) unit test that provokes/creates
the race condition explicitly (and reliably).

Closes scylladb/scylladb#27998

(cherry picked from commit a7cdb602e1)

Closes scylladb/scylladb#28097
2026-01-16 16:21:22 +02:00
Łukasz Paszkowski
1284c18647 load_sketch: Allow populating load_sketch with normalized current load
Currently, tablet allocation intentionally ignores current load (
introduced by the commit #1e407ab) which could cause identical shard
selection when allocating a small number of tablets in the same topology.
When a tablet allocator is asked to allocate N tablets (where N is smaller
than the number of shards on a node), it selects the first N lowest shards.
If multiple such tables are created, each allocator run picks the same
shards, leading to tablet imbalance across shards.

This change initializes the load sketch with the current shard load,
scaled into the [0,1] range, ensuring allocation still remains even
while starting from globally least-loaded shards.

Fixes https://github.com/scylladb/scylladb/issues/27620

Closes https://github.com/scylladb/scylladb/pull/27802

Closes scylladb/scylladb#28106
2026-01-16 13:50:16 +01:00
Patryk Jędrzejczak
d08045bb61 test: test_group0_schema_versioning: wait for schema sync in system.local
`test_schema_versioning_with_recovery` is currently flaky. It performs
a write with CL=ALL and then checks if the schema version is the same on
all nodes by calling `verify_table_versions_synced`. All nodes are expected
to sync their schema before handling the replica write. The node in
RECOVERY mode should do it through a schema pull, and other nodes should do
it through a group 0 read barrier.

The problem is in `verify_local_schema_versions_synced` that compares the
schema versions in `system.local`. The node in RECOVERY mode updates the
schema version in `system.local` after it acknowledges the replica write
as completed. Hence, the check can fail.

We fix the problem by making the function wait until the schema versions
match.

Note that RECOVERY mode is about to be retired together with the whole
gossip-based topology in 2026.2. So, this test is about to be deleted.
However, we still want to fix it, so that it doesn't bother us in older
branches.

Fixes #23803

Closes scylladb/scylladb#28114

(cherry picked from commit 6b5923c64e)

Closes scylladb/scylladb#28175
2026-01-16 11:20:45 +01:00
Sergey Zolotukhin
abcf02cbda test: disable test_start_bootstrapped_with_invalid_seed
The test intermittently fails when an invalid DNS name is resolved,
likely due to ISP DNS error hijacking (see scylladb/scylladb#28153).

Disable this test to unblock CI.

Fixes scylladb/scylladb#28153

Closes scylladb/scylladb#28162

(cherry picked from commit 799d837295)
2026-01-15 17:55:35 +02:00
Avi Kivity
fb15c818f6 Update seastar submodule (accept unbounded recursion)
* seastar f61814a489...167e47bcce (1):
  > net: posix_server_socket_impl: coroutinize accept(), fix unbounded recursion

Fixes #28166
2026-01-15 14:35:25 +02:00
Jenkins Promoter
642317d217 Update pgo profiles - aarch64 2026-01-15 05:33:48 +02:00
Jenkins Promoter
022a2f9e8d Update pgo profiles - x86_64 2026-01-15 04:55:51 +02:00
Patryk Jędrzejczak
6a36195f90 Merge '[Backport 2025.3] raft topology: preserve IP -> ID mapping of a replacing node on restart' from Scylladb[bot]
We currently do it only for a bootstrapping node, which is a bug. The
missing IP can cause an internal error, for example, in the following
scenario:
- replace fails during streaming,
- all live nodes are shut down before the rollback of replace completes,
- all live nodes are restarted,
- live nodes start hitting internal error in all operations that
  require IP of the replacing node (like client requests or REST API
  requests coming from nodetool).

We fix the bug here, but we do it separately for replace with different
IP and replace with the same IP.

For replace with different IP, we persist the IP -> host ID mapping
in `system.peers` just like for bootstrap. That's necessary, since there
is no other way to determine IP of the replacing node on restart.

For replace with the same IP, we can't do the same. This would require
deleting the row corresponding to the node being replaced from
`system.peers`. That's fine in theory, as that node is permanently
banned, so its IP shouldn't be needed. Unfortunately, we have many
places in the code where we assume that IP of a topology member is always
present in the address map or that a topology member is always present in
the gossiper endpoint set. Examples of such places:
- nodetool operations,
- REST API endpoints,
- `db::hints::manager::store_hint`,
- `group0_voter_handler::update_nodes`.

We could fix all those places and verify that drivers work properly when
they see a node in the token metadata, but not in `system.peers`.
However, that would be too risky to backport.

We take a different approach. We recover IP of the replacing node on
restart based on the state of the topology state machine and
`system.peers` just after loading `system.peers`.

We rely on the fact that group 0 is set up at this point. The only case
where this assumption is incorrect is a restart in the Raft-based
recovery procedure. However, hitting this problem then seems improbable,
and even if it happens, we can restart the node again after ensuring
that no client and REST API requests come before replace is rolled back
on the new topology coordinator. Hence, it's not worth to complicate the
fix (by e.g. looking at the persistent topology state instead of the
in-memory state machine).

Fixes #28057

Backport this PR to all branches as it fixes a problematic bug.

- (cherry picked from commit fc4c2df2ce)

- (cherry picked from commit 4526dd93b1)

- (cherry picked from commit 749b0278e5)

- (cherry picked from commit 0fed9f94f8)

Manually cherry-picked:
- 90b5b2c5f5
- 92b165b8c0

Parent PR: #27435

Closes scylladb/scylladb#28098

* https://github.com/scylladb/scylladb:
  gossiper: add_saved_endpoint: make generations of excluded nodes negative
  test: introduce test_full_shutdown_during_replace
  utils: error_injection: allow aborting wait_for_message
  raft topology: preserve IP -> ID mapping of a replacing node on restart
  pylib/rest_client.py: encode injection name
  utils/error_injection: allow to abort `injection_handler::wait_for_message()`
2026-01-14 10:04:38 +01:00
Michał Jadwiszczak
2a45a98262 docs/dev/service_levels: update docs to service levels on raft
Since Scylla 6.0, service levels are manged by Raft group0.
This patch updates table name used by service levels and adds a
paragraph describing service levels on raft.

Fixes scylladb/scylladb#18177

Closes scylladb/scylladb#26556

(cherry picked from commit 649efd198f)

Closes scylladb/scylladb#28129
2026-01-13 19:32:27 +01:00
Patryk Jędrzejczak
976f16dbaf gossiper: add_saved_endpoint: make generations of excluded nodes negative
The explanation is in the new comment in `gossiper::add_saved_endpoint`.

We add a test for this change. It's "extremely white-box", but it's better
than nothing.

(cherry picked from commit 0fed9f94f8)
2026-01-13 16:03:22 +01:00
Patryk Jędrzejczak
eb17a8d940 test: introduce test_full_shutdown_during_replace
(cherry picked from commit 749b0278e5)
2026-01-13 16:03:22 +01:00
Patryk Jędrzejczak
810c7b436e utils: error_injection: allow aborting wait_for_message
The test added in the following commit utilizes it.

(cherry picked from commit 4526dd93b1)
2026-01-13 16:03:22 +01:00
Patryk Jędrzejczak
53623205ee raft topology: preserve IP -> ID mapping of a replacing node on restart
We currently do it only for a bootstrapping node, which is a bug. The
missing IP can cause an internal error, for example, in the following
scenario:
- replace fails during streaming,
- all live nodes are shut down before the rollback of replace completes,
- all live nodes are restarted,
- live nodes start hitting internal error in all operations that
  require IP of the replacing node (like client requests or REST API
  requests coming from nodetool).

We fix the bug here, but we do it separately for replace with different
IP and replace with the same IP.

For replace with different IP, we persist the IP -> host ID mapping
in `system.peers` just like for bootstrap. That's necessary, since there
is no other way to determine IP of the replacing node on restart.

For replace with the same IP, we can't do the same. This would require
deleting the row corresponding to the node being replaced from
`system.peers`. That's fine in theory, as that node is permanently
banned, so its IP shouldn't be needed. Unfortunately, we have many
places in the code where we assume that IP of a topology member is always
present in the address map or that a topology member is always present in
the gossiper endpoint set. Examples of such places:
- nodetool operations,
- REST API endpoints,
- `db::hints::manager::store_hint`,
- `group0_voter_handler::update_nodes`.

We could fix all those places and verify that drivers work properly when
they see a node in the token metadata, but not in `system.peers`.
However, that would be too risky to backport.

We take a different approach. We recover IP of the replacing node on
restart based on the state of the topology state machine and
`system.peers` just after loading `system.peers`.

We rely on the fact that group 0 is set up at this point. The only case
where this assumption is incorrect is a restart in the Raft-based
recovery procedure. However, hitting this problem then seems improbable,
and even if it happens, we can restart the node again after ensuring
that no client and REST API requests come before replace is rolled back
on the new topology coordinator. Hence, it's not worth to complicate the
fix (by e.g. looking at the persistent topology state instead of the
in-memory state machine).

(cherry picked from commit fc4c2df2ce)
2026-01-13 16:03:22 +01:00
Petr Gusev
998b4da424 pylib/rest_client.py: encode injection name
Sometimes it's convenient to use slashes in injection names,
for example my_component/my_method/my_condition. Without quote()
we get 'handler not found' error from Scylla.

(cherry picked from commit 92b165b8c0)
2026-01-13 16:03:22 +01:00
Michał Jadwiszczak
161ff63534 utils/error_injection: allow to abort injection_handler::wait_for_message()
(cherry picked from commit 90b5b2c5f5)
2026-01-13 16:03:22 +01:00
Anna Stuchlik
48712d4917 doc: add the missing patch upgrade guide for version 2025.3
This upgrade guide has been missing and must be added on branch-2025.3.
It does not belong to other branches (including master).

Fixes https://github.com/scylladb/scylladb/issues/25522

Closes scylladb/scylladb#28116
2026-01-13 10:38:01 +02:00
Michael Litvak
492cd166f9 db/view/view_update_generator: move discover_staging_sstables to start
Call discover_staging_sstables in view_update_generator::start() instead
of in the constructor, because the constructor is called during
initialization before sstables are loaded.

The initialization order was changed in 5d1f74b86a and caused this
regression. It means the view update generator won't discover staging
sstables on startup and view updates won't be generated for them. It
also causes issues in sstable cleanup.

view_update_generator::start() is called in a later stage of the
initialization, after sstable loading, so do the discovery of staging
sstables there.

Fixes scylladb/scylladb#27956

(cherry picked from commit 5077b69c06)

Closes scylladb/scylladb#28091
2026-01-12 13:31:54 +01:00
Patryk Jędrzejczak
cefe7b270f Merge '[Backport 2025.3] database: truncate_table_on_all_shards: consider can_flush on all shards' from Scylladb[bot]
Currently, database::truncate_table_on_all_shards calls the table::can_flush only on the coordinator shard
and therefore it may miss shards with dirty data if the coordinator shard happens to have empty memtables, leading to clearing the memtables with dirty data rather than flushing them.

This change fixes that by making flush safe to be called, even if the memtable list is empty, and calling it on every shard that can flush (i.e. seal_immediate_fn is engaged).

Also, change database_test::do_with_some_data is use random keys instead of hard-coded key names, to reproduce this issue with `snapshot_list_contains_dropped_tables`.

Fixes #27639

* The issue exists since forever and might cause data loss due to wrongly clearing the memtable, so it needs backport to all live versions

- (cherry picked from commit ec4069246d)

- (cherry picked from commit 5be6b80936)

- (cherry picked from commit 0342a24ee0)

- (cherry picked from commit 02ee341a03)

- (cherry picked from commit 2a803d2261)

- (cherry picked from commit 93b827c185)

- (cherry picked from commit ebd667a8e0)

Parent PR: #27643

Closes scylladb/scylladb#28071

* https://github.com/scylladb/scylladb:
  test: database_test: do_with_some_data: randomize keys
  database: truncate_table_on_all_shards: drop outdated TODO comment
  database: truncate_table_on_all_shards: consider can_flush on all shards
  memtable_list: unify can_flush and may_flush
  test: database_test: add test_flush_empty_table_waits_on_outstanding_flush
  replica: table, storage_group, compaction_group: add needs_flush
  test: database_test: do_with_some_data_in_thread: accept void callback function
2026-01-12 11:21:42 +01:00
Jenkins Promoter
f6040eca3c Update ScyllaDB version to: 2025.3.7 2026-01-11 16:11:56 +02:00
Piotr Dulikowski
4535fe7958 Merge '[Backport 2025.3] service/storage_service: update service levels cache after upgrade to v2' from Scylladb[bot]
Service levels cache is empty after upgrade to consistent topology
if no mutations are commited to `system.service_levels_v2` or rolling
restart is not done.

To fix the bug, this patch adds service levels cache reloading after
upgrading the SL data accessor to v2 in `storage_service::topology_state_load()`.

Fixes [SCYLLADB-90](https://scylladb.atlassian.net/browse/SCYLLADB-90)

This fix should be backported to all versions containing service levels on Raft.

[SCYLLADB-90]: https://scylladb.atlassian.net/browse/SCYLLADB-90?atlOrigin=eyJpIjoiNWRkNTljNzYxNjVmNDY3MDlhMDU5Y2ZhYzA5YTRkZjUiLCJwIjoiZ2l0aHViLWNvbS1KU1cifQ

- (cherry picked from commit 53d0a2b5dc)

- (cherry picked from commit be16e42cb0)

Parent PR: #27585

Closes scylladb/scylladb#28072

* github.com:scylladb/scylladb:
  service/storage_service: update service levels cache after upgrade to v2
  service/storage_service: check if service levels were already upgraded before doing migration to raft
2026-01-09 21:09:02 +01:00
Tomasz Grabiec
aca5be7c98 test: cluster: Fix NoHostAvailable error in test_not_enough_token_owners
The driver must see server_c before we stop server_a, otherwise
there will be no live host in the pool when we attempt to drop
the keyspace:

```
   @pytest.mark.asyncio
    async def test_not_enough_token_owners(manager: ManagerClient):
        """
        Test that:
        - the first node in the cluster cannot be a zero-token node
        - removenode and decommission of the only token owner fail in the presence of zero-token nodes
        - removenode and decommission of a token owner fail in the presence of zero-token nodes if the number of token
          owners would fall below the RF of some keyspace using tablets
        """
        logging.info('Trying to add a zero-token server as the first server in the cluster')
        await manager.server_add(config={'join_ring': False},
                                 property_file={"dc": "dc1", "rack": "rz"},
                                 expected_error='Cannot start the first node in the cluster as zero-token')

        logging.info('Adding the first server')
        server_a = await manager.server_add(property_file={"dc": "dc1", "rack": "r1"})

        logging.info('Adding two zero-token servers')
        # The second server is needed only to preserve the Raft majority.
        server_b = (await manager.servers_add(2, config={'join_ring': False}, property_file={"dc": "dc1", "rack": "rz"}))[0]

        logging.info(f'Trying to decommission the only token owner {server_a}')
        await manager.decommission_node(server_a.server_id,
                                        expected_error='Cannot decommission the last token-owning node in the cluster')

        logging.info(f'Stopping {server_a}')
        await manager.server_stop_gracefully(server_a.server_id)

        logging.info(f'Trying to remove the only token owner {server_a} by {server_b}')
        await manager.remove_node(server_b.server_id, server_a.server_id,
                                  expected_error='cannot be removed because it is the last token-owning node in the cluster')

        logging.info(f'Starting {server_a}')
        await manager.server_start(server_a.server_id)

        logging.info('Adding a normal server')
        await manager.server_add(property_file={"dc": "dc1", "rack": "r2"})

        cql = manager.get_cql()

        await wait_for_cql_and_get_hosts(cql, [server_a], time.time() + 60)

>       async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2} AND tablets = { 'enabled': true }") as ks_name:

test/cluster/test_not_enough_token_owners.py:57:
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
/usr/lib64/python3.14/contextlib.py:221: in __aexit__
    await anext(self.gen)
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _

manager = <test.pylib.manager_client.ManagerClient object at 0x7f37efe00830>
opts = "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 2} AND tablets = { 'enabled': true }"
host = None

    @asynccontextmanager
    async def new_test_keyspace(manager: ManagerClient, opts, host=None):
        """
        A utility function for creating a new temporary keyspace with given
        options. It can be used in a "async with", as:
            async with new_test_keyspace(ManagerClient, '...') as keyspace:
        """
        keyspace = await create_new_test_keyspace(manager.get_cql(), opts, host)
        try:
            yield keyspace
        except:
            logger.info(f"Error happened while using keyspace '{keyspace}', the keyspace is left in place for investigation")
            raise
        else:
>           await manager.get_cql().run_async("DROP KEYSPACE " + keyspace, host=host)
E           cassandra.cluster.NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host: 127.69.108.39:9042 dc1>: ConnectionException('Pool for 127.69.108.39:9042 is shutdown')})

test/cluster/util.py:544: NoHostAvailable
```

Fixes #28011

Closes scylladb/scylladb#28040

(cherry picked from commit 34df158605)

Closes scylladb/scylladb#28070
2026-01-09 19:11:40 +01:00
Botond Dénes
16576da935 reader_concurrency_semaphore: add protection against negative count resource leaks
The semaphore has detection and protection against regular resource
leaks, where some resources go unaccounted for and are not released by
the time the semaphore is destroyed. There is no detection or protection
against negative leaks: where resources are "made up" of thin air. This
kind of leaks looks benign at first sight, a few extra resources won't
hurt anyone so long as this is a small amount. But turns out that even a
single extra count resource can defeat a very important anti-deadlock
protection in can_admit_read(): the special case which admits a new
permit regardless of memory resources, when all original count resources
all available. This check uses ==, so if resource > original, the
protection is defeated indefinitely. Instead of just changing == to >=,
we add detection of such negative leaks to signal(), via
on_internal_error_noexcept().
At this time I still don't now how this negative leak happens (the code
doesn't confess), with this detection, hopefully we'll get a clue from
tests or the field. Note that on_internal_error_noexcept() will not
generate a coredump, unless ScyllaDB is explicitely configured to do so.
In production, it will just generate an error log with a backtrace.
The detection also clams the _resources to _initial_resources, to
prevent any damage from the negativae leak.

I just noticed that there is no unit test for the deadlock protection
described above, so one is added in this PR, even if only loosely
related to the rest of the patch.

Fixes: SCYLLADB-163

Closes scylladb/scylladb#27764

(cherry picked from commit e4da0afb8d)

Closes scylladb/scylladb#28002
2026-01-09 13:28:14 +02:00
Benny Halevy
c495252be6 test: database_test: do_with_some_data: randomize keys
With randomized keys, and since we're inserting only 2 keys,
it is possible that they would end up owned only by a single shard,
reproducing #27639 in snapshot_list_contains_dropped_tables.

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit ebd667a8e0)
2026-01-09 08:35:29 +02:00
Benny Halevy
b5d537283d database: truncate_table_on_all_shards: drop outdated TODO comment
The comment was added in 83323e155e
Since then, table::seal_active_memtable was improved to guarantee
waiting on oustanding flushes on success (See d55a2ac762), so
we can remove this TODO comment (it also not covered by any issue
so nobody is planned to ever work on it).

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit 93b827c185)
2026-01-09 08:35:00 +02:00
Benny Halevy
fd9ad9a11c database: truncate_table_on_all_shards: consider can_flush on all shards
can_flush might return a different value for each shard
so check it right before deciding whether to flush or clear a memtable
shard.

Note that under normal condition can_flush would always return true
now that it checks only the presence of the seal memtable function
rather than check memtable_list::empty().

Fixes #27639

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit 2a803d2261)
2026-01-09 08:34:31 +02:00
Benny Halevy
4968ea4ab6 memtable_list: unify can_flush and may_flush
Now that we have a unit test proving that it's safe to flush an
empty memtable list there is no need to distinguish between
may_flush and can_flush.

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit 02ee341a03)
2026-01-09 08:31:35 +02:00
Benny Halevy
d5dcfd9133 test: database_test: add test_flush_empty_table_waits_on_outstanding_flush
Test that table::flush waits on outstanding flushes, even if the active memtable is empty

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit 0342a24ee0)
2026-01-09 08:31:33 +02:00
Benny Halevy
430059f64c replica: table, storage_group, compaction_group: add needs_flush
Table needs flush if not all its memtable lists are empty.
To be used in the next patch for a unit test.

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit 5be6b80936)
2026-01-09 08:21:13 +02:00
Benny Halevy
0e8b738dff test: database_test: do_with_some_data_in_thread: accept void callback function
Many test cases already assume `func` is being called a seastar
thread and although the function they pass returns a (ready) future,
it serves no purpose other than to conform to the interface.

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>
(cherry picked from commit ec4069246d)
2026-01-09 08:16:41 +02:00
Michał Jadwiszczak
c0a7b928ed service/storage_service: update service levels cache after upgrade to v2
Service levels cache is empty after upgrade to consistent topology
if no mutations are commited to `system.service_levels_v2` or rolling
restart is not done.

To fix the bug, this commit adds service levels cache reloading after
upgrading the SL data accessor to v2 in `storage_service::topology_state_load()`.

Fixes SCYLLADB-90

(cherry picked from commit be16e42cb0)
2026-01-08 22:45:59 +00:00
Michał Jadwiszczak
ff01aa3548 service/storage_service: check if service levels were already upgraded
before doing migration to raft

There is no need to call `service_level_controller::upgrade_to_v2()`
on every topology state load, we only need to do it once.

(cherry picked from commit 53d0a2b5dc)
2026-01-08 22:45:59 +00:00
Anna Stuchlik
adf2d0efd4 doc: remove cassandra-stress from installation instructions
The cassandra-stress tool is no longer part of the default package
and cannot be run in the way described.

This commit removes the instruction to run cassandra-stress.

Fixes https://github.com/scylladb/scylladb/issues/24994

Closes scylladb/scylladb#27726

(cherry picked from commit 624869de86)

Closes scylladb/scylladb#27950
2026-01-08 18:01:04 +02:00
Benny Halevy
9f733ceaee db: system_keyspace: get_group0_history: unfreeze_gently
Prevent stall when the group0 history is too long using unfreeze_gently
rather than the synchronous unfreeze() function

Fixes #27872

Signed-off-by: Benny Halevy <bhalevy@scylladb.com>

Closes scylladb/scylladb#27873

(cherry picked from commit f60033db63)

Closes scylladb/scylladb#27908
2026-01-08 18:00:13 +02:00