The test intentionally creates huge index pages.
But since 5e7fb08bf3,
the index reader allocates a block of memory for a whole index page,
instead of incrementally allocating small pieces during index parsing.
This giant allocation causes the test to fail spuriously in CI sometimes.
Fix this by disabling sstable compression on the test table,
which puts a hard cap of 2000 keys per index page.
Fixes: SCYLLADB-1152
Closesscylladb/scylladb#29152
(cherry picked from commit f29525f3a6)
Closesscylladb/scylladb#29172Closesscylladb/scylladb#29259
The removenove initiator could have an outdated token ring (still considering
the node removed by the previous removenode a token owner) and unexpectedly
reject the operation.
Fix that by waiting for token ring and group0 consistency before removenode.
Note that the test already checks that consistency, but only for one node,
which is different from the removenode initiator.
This test has been removed in master together with the code being tested
(the gossip-based topology). Hence, the fix is submitted directly to 2026.1.
Fixes https://scylladb.atlassian.net/browse/SCYLLADB-1103
Backport to all supported branches (other than 2026.1), as the test can fail
there.
Closesscylladb/scylladb#29108
(cherry picked from commit 1398a55d16)
Closesscylladb/scylladb#29205
Move all ${{ }} expression interpolations into env: blocks so they are
passed as environment variables instead of being expanded directly into
shell scripts. This prevents an attacker from escaping the heredoc in
the Validate Comment Trigger step and executing arbitrary commands on
the runner.
The Verify Org Membership step is hardened in the same way for
defense-in-depth.
Refs: GHSA-9pmq-v59g-8fxp
Fixes: SCYLLADB-954
Closesscylladb/scylladb#28935
(cherry picked from commit 977bdd6260)
Closesscylladb/scylladb#28947
nodetool cluster repair without additional params repairs all tablet
keyspaces in a cluster. Currently, if a table is dropped while
the command is running, all tables are repaired but the command finishes
with a failure.
Modify nodetool cluster repair. If a table wasn't specified
(i.e. all tables are repaired), the command finishes successfully
even if a table was dropped.
If a table was specified and it does not exist (e.g. because it was
dropped before the repair was requested), then the behavior remains
unchanged.
Fixes: SCYLLADB-568.
Closesscylladb/scylladb#28739
(cherry picked from commit 2e68f48068)
Closesscylladb/scylladb#29006Closesscylladb/scylladb#29038
mv: allow skipping view updates when a collection is unmodified
When we generate view updates, we check whether we can skip the
entire view update if all columns selected by the view are unmodified.
However, for collection columns, we only check if they were unset
before and after the update.
In this patch we add a check for the actual collection contents.
We perform this check for both virtual and non-virtual selections.
When the column is only a virtual column in the view, it would be
enough to check the liveness of each collection cell, however for
that we'd need to deserialize the entire collection anyway, which
should be effectively as expensive as comparing all of its bytes.
Fixes: SCYLLADB-996
- (cherry picked from commit 01ddc17ab9)
Parent PR: #28839Closesscylladb/scylladb#28977
* github.com:scylladb/scylladb:
Merge 'mv: allow skipping view updates when a collection is unmodified' from Wojciech Mitros
mv: remove dead code in view_updates::can_skip_view_updates
Closesscylladb/scylladb#29094
`test_raft_no_quorum.py::test_cannot_add_new_node` is currently flaky in dev
mode. The bootstrap of the first node can fail due to `add_entry()` timing
out (with the 1s timeout set by the test case).
Other test cases in this test file could fail in the same way as well, so we
need a general fix. We don't want to increase the timeout in dev mode, as it
would slow down the test. The solution is to keep the timeout unchanged, but
set it only after quorum is lost. This prevents unexpected timeouts of group0
operations with almost no impact on the test running time.
A note about the new `update_group0_raft_op_timeout` function: waiting for
the log seems to be necessary only for
`test_quorum_lost_during_node_join_response_handler`, but let's do it
for all test cases just in case (including `test_can_restart` that shouldn't
be flaky currently).
Fixes https://scylladb.atlassian.net/browse/SCYLLADB-913Closesscylladb/scylladb#28998
(cherry picked from commit 526e5986fe)
Closesscylladb/scylladb#29068Closesscylladb/scylladb#29097
Set enable_schema_commitlog for each group0 tables.
Assert that group0 tables use schema commitlog in ensure_group0_schema
(per each command).
Fixes: https://scylladb.atlassian.net/browse/SCYLLADB-914.
Needs backport to all live releases as all are vulnerable
Closesscylladb/scylladb#28876
* github.com:scylladb/scylladb:
test: add test_group0_tables_use_schema_commitlog
db: service: remove group0 tables from schema commitlog schema initializer
service: ensure that tables updated via group0 use schema commitlog
db: schema: remove set_is_group0_table param
(cherry picked from commit b90fe19a42)
Closesscylladb/scylladb#28916Closesscylladb/scylladb#28986
The default 100ms timeout for client readiness in tests is too
aggressive. In some test environments, this is not enough time for
client creation, which involves address resolution and TLS certificate
reading, leading to flaky tests.
This commit increases the default client creation timeout to 10 seconds.
This makes the tests more robust, especially in slower execution
environments, and prevents similar flakiness in other test cases.
Fixes: VECTOR-547
Fixes: SCYLLADB-802
Fixes: SCYLLADB-825
Fixes: SCYLLADB-826
Backport to 2025.4 and 2026.1, as the same problem occurs on these branches and can potentially make the CI flaky there as well.
- (cherry picked from commit bf369326d6)
Parent PR: #28879Closesscylladb/scylladb#28895
* github.com:scylladb/scylladb:
vector_search: test: include ANN error in assertion
vector_search: test: fix HTTPS client test flakiness
When the test fails, the assertion message does not include
the error from the ANN request.
This change enhances the assertion to include the specific ANN error,
making it easier to diagnose test failures.
The default 100ms timeout for client readiness in tests is too
aggressive. In some test environments, this is not enough time for
client creation, which involves address resolution and TLS certificate
reading, leading to flaky tests.
This commit increases the default client creation timeout to 10 seconds.
This makes the tests more robust, especially in slower execution
environments, and prevents similar flakiness in other test cases.
Fixes: VECTOR-547, SCYLLADB-802
The test is currently flaky with `reuse_ip = True`. The issue is that the
test retries replace before the first replace is rolled back and the
first replacing node is removed from gossip. The second replacing node
can see the entry of the first replacing node in gossip. This entry has
a newer generation than the entry of the node being replaced, and both
replacing nodes have the same IP as the node being replaced. Therefore,
the second replacing node incorrectly considers this entry as the entry
of the node being replaced. This entry is missing rack and DC, so the
second replace fails with
```
ERROR 2026-02-24 21:19:03,420 [shard 0:main] init - Startup failed:
std::runtime_error (Cannot replace node
8762a9d2-3b30-4e66-83a1-98d16c5dd007/127.61.127.1 with a node on
a different data center or rack.
Current location=UNKNOWN_DC/UNKNOWN_RACK, new location=dc1/rack2)
```
Fixes SCYLLADB-805
Closesscylladb/scylladb#28829
(cherry picked from commit ba7f314cdc)
Closesscylladb/scylladb#28953
This commit updates the documentation for the unified installer.
- The Open Source example is replaced with version 2025.1 (Source Available, currently supported, LTS).
- The info about CentOS 7 is removed (no longer supported).
- Java 8 is removed.
- The example for cassandra-stress is removed (as it was already removed on other installation pages).
Fixes https://github.com/scylladb/scylladb/issues/28150Closesscylladb/scylladb#28152
(cherry picked from commit 855c503c63)
Closesscylladb/scylladb#28910Closesscylladb/scylladb#28927
scylladb/scylla container image doesn't include systemctl binary, while it
is used by perftune.py script shipped within the same image.
Scylla Operator runs this script to tune Scylla nodes/containers,
expecting its all dependencies to be available in the container's PATH.
Without systemctl, the script fails on systems that run irqbalance
(e.g., on EKS nodes) as the script tries to reconfigure irqbalance and
restart it via systemctl afterwards.
Fixes: scylladb/scylla-operator#3080Closesscylladb/scylladb#28567
(cherry picked from commit b4f0eb666f)
Closesscylladb/scylladb#28845
(cherry picked from commit 4cc5c2605f)
Closesscylladb/scylladb#28890
Consider the following scenario:
1. Let nodes A,B,C form a cluster with RF=3
2. Write query with CL=QUORUM is submitted and is acknowledged by
nodes B,C
3. Follow-up read query with CL=QUORUM is sent to verify the write
from the previous step
4. Coordinator sends data/digest requests to the nodes A,B. Since the
node A is missing data, digest mismatches and data reconciliation
is triggered
5. The node A or B fails, becomes unavailable, etc
6. During reconciliation, data requests are sent to node A,B and fail
failing the entire read query
When the above scenario happens, the tests using `start_writes()` fail
with the following stacktrace:
```
...
> await finish_writes()
test/cluster/test_tablets_migration.py:259:
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
test/pylib/util.py:241: in finish
await asyncio.gather(*tasks)
test/pylib/util.py:227: in do_writes
raise e
_ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
worker_id = 1
...
> rows = await cql.run_async(rd_stmt, [pk])
E cassandra.ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] message="Operation failed for test_1767777001181_bmsvk.test - received 1 responses and 1 failures from 2 CL=QUORUM." info={'consistency': 'QUORUM', 'required_responses': 2, 'received_responses': 1, 'failures': 1}
```
Note that when a node failure happens before/during a read query,
there is no test failure as the speculative retries are enabled
by default. Hence an additional data/digest read is sent to the third
remaining node.
However, the same speculative read is cancelled the moment, the read
query reaches CL which may trigger a read-repair.
This change:
- Retries the verification read in start_writes() on failure to mitigate
races between reads and node failures
- Adds additional logging to correlate Python exceptions with Scylla logs
Fixes https://github.com/scylladb/scylladb/issues/27478
Fixes https://github.com/scylladb/scylladb/issues/27974
Fixes https://github.com/scylladb/scylladb/issues/27494
Fixes https://github.com/scylladb/scylladb/issues/23529
Note that this change test flakiness observed during tablet transitions.
However, it serves as a workaround for a higher-level issue
https://github.com/scylladb/scylladb/issues/28125Closesscylladb/scylladb#28140
(cherry picked from commit e07fe2536e)
Closesscylladb/scylladb#28825
The futurization refactoring in 9d3755f276 ("replica: Futurize
retrieval of sstable sets in compaction_group_view") changed
maybe_wait_for_sstable_count_reduction() from a single predicated
wait:
```
co_await cstate.compaction_done.wait([..] {
return num_runs_for_compaction() <= threshold
|| !can_perform_regular_compaction(t);
});
```
to a while loop with a predicated wait:
```
while (can_perform_regular_compaction(t)
&& co_await num_runs_for_compaction() > threshold) {
co_await cstate.compaction_done.wait([this, &t] {
return !can_perform_regular_compaction(t);
});
}
```
This was necessary because num_runs_for_compaction() became a
coroutine (returns future<size_t>) and can no longer be called
inside a condition_variable predicate (which must be synchronous).
However, the inner wait's predicate — !can_perform_regular_compaction(t)
— only returns true when compaction is disabled or the table is being
removed. During normal operation, every signal() from compaction_done
wakes the waiter, the predicate returns false, and the waiter
immediately goes back to sleep without ever re-checking the outer
while loop's num_runs_for_compaction() condition.
This causes memtable flushes to hang forever in
maybe_wait_for_sstable_count_reduction() whenever the sstable run
count exceeds the threshold, because completed compactions signal
compaction_done but the signal is swallowed by the predicate.
Fix by replacing the predicated wait with a bare wait(), so that
any signal (including from completed compactions) causes the outer
while loop to re-evaluate num_runs_for_compaction().
Fixes: https://scylladb.atlassian.net/browse/SCYLLADB-610Closesscylladb/scylladb#28801
(cherry picked from commit bb57b0f3b7)
`test_autoretrain_dict` sporadically fails because the default
compression algorithm was changed after the test was written.
`9ffa62a986815709d0a09c705d2d0caf64776249` was an attempt to fix it by
changing the compression configuration during node startup. However,
the configuration change had an incorrect YAML format and was
ignored by ScyllaDB. This commit fixes it.
Fixes: scylladb/scylladb#28204Closesscylladb/scylladb#28746
(cherry picked from commit cd4caed3d3)
Closesscylladb/scylladb#28792
This commit removes the information that Alternator doesn't support tablets.
The limitation is no longer valid.
Fixes SCYLLADB-778
Closesscylladb/scylladb#28781
(cherry picked from commit e2333a57ad)
Closesscylladb/scylladb#28793
This workflow calls the reusable backport-with-jira workflow from
scylladb/github-automation to enable automatic backport PR creation with
Jira sub-issue integration.
The workflow triggers on:
- Push to master/next-*/branch-* branches (for promotion events)
- PR labeled with backport/X.X pattern (for manual backport requests)
- PR closed/merged on version branches (for chain backport processing)
Features enabled by calling the shared workflow:
- Creates Jira sub-issues under the main issue for each backport version
- Sorts versions descending (highest first: 2025.4 -> 2025.3 -> 2025.2)
- Cherry-picks from previous version branch to avoid repeated conflicts
- On Jira API failure: adds comment to main issue, applies 'jira-sub-issue-creation-failed' label, continues with PR
Closesscylladb/scylladb#28804
(cherry picked from commit b211590bc0)
Closesscylladb/scylladb#28813
The connection's `cpu_concurrency_t` struct tracks the state of a connection
to manage the admission of new requests and prevent CPU overload during
connection storms. When a connection holds units (allowed only 0 or 1), it is
considered to be in the "CPU state" and contributes to the concurrency limits
used when accepting new connections.
The bug stems from the fact that `counted_data_source_impl::get` and
`counted_data_sink_impl::put` calls can interleave during execution. This
occurs because of `should_parallelize` and `_ready_to_respond`, the latter being
a future chain that can run in the background while requests are being read.
Consequently, while reading request (N), the system may concurrently be
writing the response for request (N-1) on the same connection.
This interleaving allows `return_all()` to be called twice before the
subsequent `consume_units()` is invoked. While the second `return_all()` call
correctly returns 0 units, the matching `consume_units()` call would
mistakenly take an extra unit from the semaphore. Over time, a connection
blocked on a read operation could end up holding an unreturned semaphore
unit. If this pattern repeats across multiple connections, the semaphore
units are eventually depleted, preventing the server from accepting any
new connections.
The fix ensures that we always consume the exact number of units that were
previously returned. With this change, interleaved operations behave as
follows:
get() return_all — returns 1 unit
put() return_all — returns 0 units
get() consume_units — takes back 1 unit
put() consume_units — takes back 0 units
Logically, the networking phase ends when the first network operation
concludes. But more importantly, when a network operation
starts, we no longer hold any units.
Other solutions are possible but the chosen one seems to be the
simplest and safest to backport.
Fixes SCYLLADB-485
Backport: all supported affected versions, bug introduced with initial feature implementation in: ed3e4f33fd
- (cherry picked from commit 0376d16ad3)
- (cherry picked from commit 3b98451776)
Parent PR: #28530Closesscylladb/scylladb#28715
* github.com:scylladb/scylladb:
test: decrease strain in test_startup_response
test: auth_cluster: add test for hanged AUTHENTICATING connections
transport: fix connection code to consume only initially taken semaphore units
transport: remove redundant futurize_invoke from counted data sink and source
For 2025.3 and 2025.4 this test runs order of magnitude
slower in debug mode. Potentially due to passwords::check
running in alien thread and overwhelming the CPU (this is
fixed in newer versions).
Decreasing the number of connections in test makes it fast
again, without breaking reproducibility.
As additional measure we double the timeout.
The connection's cpu_concurrency_t struct tracks the state of a connection
to manage the admission of new requests and prevent CPU overload during
connection storms. When a connection holds units (allowed only 0 or 1), it is
considered to be in the "CPU state" and contributes to the concurrency limits
used when accepting new connections.
The bug stems from the fact that `counted_data_source_impl::get` and
`counted_data_sink_impl::put` calls can interleave during execution. This
occurs because of `should_parallelize` and `_ready_to_respond`, the latter being
a future chain that can run in the background while requests are being read.
Consequently, while reading request (N), the system may concurrently be
writing the response for request (N-1) on the same connection.
This interleaving allows `return_all()` to be called twice before the
subsequent `consume_units()` is invoked. While the second `return_all()` call
correctly returns 0 units, the matching `consume_units()` call would
mistakenly take an extra unit from the semaphore. Over time, a connection
blocked on a read operation could end up holding an unreturned semaphore
unit. If this pattern repeats across multiple connections, the semaphore
units are eventually depleted, preventing the server from accepting any
new connections.
The fix ensures that we always consume the exact number of units that were
previously returned. With this change, interleaved operations behave as
follows:
get() return_all — returns 1 unit
put() return_all — returns 0 units
get() consume_units — takes back 1 unit
put() consume_units — takes back 0 units
Logically, the networking phase ends when the first network operation
concludes. But more importantly, when a network operation
starts, we no longer hold any units.
Other solutions are possible but the chosen one seems to be the
simplest and safest to backport.
Fixes SCYLLADB-485
(cherry picked from commit 0376d16)
- Correct `calc_part_size` function since it could return more than 10k parts
- Add tests
- Add more checks in `calc_part_size` to comply with S3 limits
Fixes: https://scylladb.atlassian.net/browse/SCYLLADB-640
Must be ported back to 2025.3/4 and 2026.1 since we may encounter this bug in production clusters
- (cherry picked from commit 289e910cec)
- (cherry picked from commit 6280cb91ca)
- (cherry picked from commit 960adbb439)
Parent PR: #28592Closesscylladb/scylladb#28696
* github.com:scylladb/scylladb:
s3_client: add more constrains to the calc_part_size
s3_client: add tests for calc_part_size
s3_client: correct multipart part-size logic to respect 10k limit
Hints destined for some other node can only be drained after the other node is no longer a replica of any vnode or tablet. In case when tablets are present, a node might still technically be a replica of some tablets after it moved to left state. When it no longer is a replica of any tablet, it becomes "released" and storage service generates a notification about it. Hinted handoff listens to this notification and kicks off draining hints after getting it.
The current implementation of the "released" notification would trigger every time raft topology state is reloaded and a left node without any tokens is present in the raft topology. Although draining hints is idempotent, generating duplicate notifications is wasteful and recently became very noisy after in 44de563 verbosity of the draining-related log messages have been increased. The verbosity increase itself makes sense as draining is supposed to be a rare operation, but the duplicate notification bug now needs to be addressed.
Fix the duplicate notification problem by passing the list of previously released nodes to the `storage_service::raft_topology_update_ip` function and filtering based on it. If this function processes the topology state for the first time, it will not produce any notifications. This is fine as hinted handoff is prepared to detect "released" nodes during the startup sequence in main.cc and start draining the hints there, if needed.
Fixes: scylladb/scylladb#28301
Refs: scylladb/scylladb#25031
The log messages added in 44de563 cause a lot of noise during topology operations and tablet migrations, so the fix should be backported to all affected versions (2025.4 and 2026.1).
- (cherry picked from commit 10e9672852)
- (cherry picked from commit d28c841fa9)
- (cherry picked from commit 29da20744a)
Parent PR: #28367Closesscylladb/scylladb#28611
* github.com:scylladb/scylladb:
storage_service: fix indentation after previous patch
raft topology: generate notification about released nodes only once
raft topology: extract "released" nodes calculation to external function
Hints destined for some other node can only be drained after the other
node is no longer a replica of any vnode or tablet. In case when tablets
are present, a node might still technically be a replica of some tablets
after it moved to left state. When it no longer is a replica of any
tablet, it becomes "released" and storage service generates a
notification about it. Hinted handoff listens to this notification and
kicks off draining hints after getting it.
The current implementation of the "released" notification would trigger
every time raft topology state is reloaded and a left node without any
tokens is present in the raft topology. Although draining hints is
idempotent, generating duplicate notifications is wasteful and recently
became very noisy after in 44de563 verbosity of the draining-related log
messages have been increased. The verbosity increase itself makes sense
as draining is supposed to be a rare operation, but the duplicate
notification bug now needs to be addressed.
Fix the duplicate notification problem by passing the list of previously
released nodes to the `storage_service::raft_topology_update_ip`
function and filtering based on it. If this function processes the
topology state for the first time, it will not produce any
notifications. This is fine as hinted handoff is prepared to detect
"released" nodes during the startup sequence in main.cc and start
draining the hints there, if needed.
Fixes: #28301
Refs: #25031
(cherry picked from commit d28c841fa9)
In the following commits we will need to compare the set of released
nodes before and after reload of raft topology state. Moving the logic
that calculates such a set to a separate function will make it easier to
do.
(cherry picked from commit 10e9672852)
Add --blocked-reactor-notify-ms argument to allow overriding the default
blocked reactor notification timeout value of 25 ms.
This change provides users the flexibility to customize the reactor
notification timeout as needed.
Fixes: scylladb/scylla-enterprise#5525Closesscylladb/scylladb#26892
(cherry picked from commit a05ebbbfbb)
Closesscylladb/scylladb#26971
Paxos state tables are internal tables fully managed by Scylla
and they shouldn't be exposed to the user nor they shouldn't be backed up.
This commit hides those kind of tables from all listings and if such table
is directly described with `DESC ks."tbl$paxos"`, the description is generated
withing a comment and a note for the user is added.
Fixes https://github.com/scylladb/scylladb/issues/28183
LWT on tablets and paxos state tables are present in 2025.4, so the patch should be backported to this version.
- (cherry picked from commit f89a8c4ec4)
- (cherry picked from commit 9baaddb613)
Parent PR: #28230Closesscylladb/scylladb#28507
* github.com:scylladb/scylladb:
test/cqlpy: add reproducer for hidden Paxos table being shown by DESC
cql3/statements/describe_statement: hide paxos state tables
When calling a migration notification from the context of a notification
callback, this could lead to a deadlock with unregistering a listener:
A: the parent notification is called. it calls thread_for_each, where it
acquires a read lock on the vector of listeners, and calls the
callback function for each listener while holding the lock.
B: a listener is unregistered. it calls `remove` and tries to acquire a
write lock on the vector of listeners. it waits because the lock is
held.
A: the callback function calls another notification and calls
thread_for_each which tries to acquire the read lock again. but it
waits since there is a waiter.
Currently we have such concrete scenario when creating a table, where
the callback of `before_create_column_family` in the tablet allocator
calls `before_allocate_tablet_map`, and this could deadlock with node
shutdown where we unregister listeners.
Fix this by not acquiring the read lock again in the nested
notification. There is no need because the read lock is already held by
the parent notification while the child notification is running. We add
a function `thread_for_each_nested` that is similar to `thread_for_each`
except it assumes the read lock is already held and doesn't acquire it,
and it should be used for nested notifications instead of
`thread_for_each`.
Fixesscylladb/scylladb#27364Closesscylladb/scylladb#27637
(cherry picked from commit 55f4a2b754)
Closesscylladb/scylladb#28557