Migrate the `system_distributed.view_build_status` table to `system.view_build_status_v2`. The writes to the v2 table are done via raft group0 operations.
The new parameter `view_builder_version` stored in `scylla_local` indicates whether nodes should use the old or the new table.
New clusters use v2. Otherwise, the migration to v2 is initiated by the topology coordinator when the feature is enabled. It reads all the rows from the old table and writes them to the new table, and sets `view_builder_version` to v2. When the change is applied, all view_builder services are updated to write and read from the v2 table.
The old table `system_distributed.view_build_status` is set to read virtually from the new table in order to maintain compatibility.
When removing a node from the cluster, we remove its rows from the table atomically (fixes https://github.com/scylladb/scylladb/issues/11836). Also, during the migration, we remove all invalid rows.
Fixesscylladb/scylladb#15329
dtest https://github.com/scylladb/scylla-dtest/pull/4827Closesscylladb/scylladb#19745
* github.com:scylladb/scylladb:
view: test view_build_status table with node replace
test/pylib: use view_build_status_v2 table in wait_for_view
view_builder: common write view_build_status function
view_builder: improve migration to v2 with intermediate phase
view: delete node rows from view_build_status on node removal
view: sanitize view_build_status during migration
view: make old view_build_status table a virtual table
replica: move streaming_reader_lifecycle_policy to header file
view_builder: test view_build_status_v2
storage_service: add view_build_status to raft snapshot
view_builder: migration to v2
db:system_keyspace: add view_builder_version to scylla_local
view_builder: read view status from v2 table
view_builder: introduce writing status mutations via raft
view_builder: pass group0_client and qp to view_builder
view_builder: extract sys_dist status operations to functions
db:system_keyspace: add view_build_status_v2 table
In test.py every asyncio task spawned during the test must be finished before the next test, otherwise, tests might affect each other results.
The developers are responsible for writing asyncio code in a way that doesn’t leave task objects unfinished.
Test.py has a mechanism that helps test writers avoid such tasks. At the end of each test case, it verifies that the test did not produce/leave any tasks and sets an event object that fails the next test at the start if this is the case(issue https://github.com/scylladb/scylladb/issues/16472)
The problem with this was that breaking the next test was counterintuitive, and the logging for this situation was insufficient and unobvious.
notes: Task.cancel() is not an option to avoid task leakage
1) Calling cancel() Does Not Cancel The Task : the cancel() method just request that the target task cancel.
2) Calling cancel() Does Not Block Until The Task is Cancelled: If the caller needs to know the task is cancelled and done, it could await for the target
3) In particular PR, task.cancel() cancell task on client(ManagerClient) but not on http server(ScyllaManager). so "await" is needed.
Closesscylladb/scylladb#20012
Any expired tombstone can be garbage collected if it doesn't shadow data in the commit log, memtable, or uncompacting SSTables.
This PR introduces a new mode to major compaction, enabled by the `consider_only_existing_data` flag that bypasses these checks. When enabled, memtables and old commitlog segments are cleared with a system-wide flush and all the sstables (after flush) are included in the compaction, so that it works with all data generated up to a given time point.
This new mode works with the assumption that newly written data will not be shadowed by expired tombstones. So it ignores new sstables (and new data written to memtable) created after compaction started. Since there was a system wide flush, commitlog checks can also be skipped when garbage collecting tombstones. Introducing data shadowed by a tombstone during compaction can lead to undefined behavior, even without this PR, as the tombstone may or may not have already been garbage collected.
Fixes#19728Closesscylladb/scylladb#20031
* github.com:scylladb/scylladb:
cql-pytest: add test to verify consider_only_existing_data compaction option
tools/scylla-nodetool: add consider-only-existing-data option to compact command
api: compaction: add `consider_only_existing_data` option
compaction: consider gc_check_only_compacting_sstables when deducing max purgeable timestamp
compaction: do not check commitlog if gc_check_only_compacting_sstables is enabled
tombstone_gc_state: introduce with_commitlog_check_disabled()
compaction: introduce new option to check only compacting sstables for gc
compaction: rename maybe_flush_all_tables to maybe_flush_commitlog
compaction: maybe_flush_all_tables: add new force_flush param
To enhance the test reports UX:
1. switching off/on passed/failed/skipped test for better visibility
2. better searching in test results
3. understanding the trends of execution for each test
4. better configurability of the final report
Enable allure adapter for all python tests.
Add tags and parameters to the test to be able to distinguish them across modes and runs.
Related: https://github.com/scylladb/qa-tasks/issues/1665
Related: https://github.com/scylladb/scylladb/pull/19335
Related: https://github.com/scylladb/scylladb/pull/18169Closesscylladb/scylladb#19942
* github.com:scylladb/scylladb:
[test.py] Clean duplicated arg for test suite
[test.py] Enable allure for python test
Allow to return from server_add() when a server reaches specified state.
One of:
- PROCESS_STARTED
- HOST_ID_QUERIED (previously called NOT_CONNECTED)
- CQL_CONNECTED (renamed from CONNECTED)
- CQL_QUERIED (was just QUERIED)
Also, rename CqlUpState to ServerUpState and move to internal_types.
Change the util function wait_for_view to read the view build status
from the system.view_build_status_v2 table which replaces
system_distributed.view_build_status.
The old table can still be used but it is less efficient because it's
implemented as a virtual table which reads from the v2 table, so it's
better to read directly from the v2 table. This can cause slowness in
tests.
The additional util function wait_for_view_v1 reads from the old table.
This may be needed in upgrade tests if the v2 table is not available
yet.
Currently if a coordinator and a node being replaced are in the same DC
while inter-dc encryption is enabled (connections between nodes in the
same DC should not be encrypted) the replace operation will fail. It
fails because a coordinator uses non encrypted connection to push raft
data to the new node, but the new node will not accept such connection
until it knows which DC the coordinator belongs to and for that the raft
data needs to be transferred.
The series adds the test for this scenario and the fix for the
chicken&egg problem above.
The series (or at least the fix itself) needs to be backported because
this is a serious regression.
Fixes: scylladb/scylladb#19025Closesscylladb/scylladb#20290
* github.com:scylladb/scylladb:
topology coordinator: fix indentation after the last patch
topology coordinator: do not add replacing node without a ring to topology
test: add test for replace in clusters with encryption enabled
test.py: add server encryption support to cluster manager
.gitignore: fix pattern for resources to match only one specific directory
The method starts a task that uses sstables_loader load-and-stream
functionality to bring new sstables into the cluster. The existing
load-and-stream picks up sstables from upload/ directory, the newly
introduced task collects them from S3 bucket and given prefix (that
correspond to the path where backup API method put them).
Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
This adds minimal implementation of the start-backup API call.
The method starts a task that uploads all files from the given keyspace's snapshot to the requested endpoint/bucket. Arguments are:
- endpoint -- the ID in object_store.yaml config file
- bucket -- the target bucket to put objects into
- keyspace -- the keyspace to work on
- snapshot -- the method assumes that the snapshot had been already taken and only copies sstables from it
The task runs in the background, its task_id is returned from the method once it's spawned and it should be used via /task_manager API to track the task execution and completion (hint: it's good to have non-zero TTL value to make sure fast backups don't finish before the caller manages to call wait_task API).
Sstables components are scanned for all tables in the keyspace and are uploaded into the /bucket/${cf_name}/${snapshot_name}/ path.
refs: #18391Closesscylladb/scylladb#19890
* github.com:scylladb/scylladb:
tools/scylla-nodetool: add backup integration
docs: Document the new backup method
test/object_store: Test that backup task is abortable
test/object_store: Add simple backup test
test/object_store: Move format_tuples()
test/pylib: Add more methods to rest client
backup-task: Make it abortable (almost)
code: Introduce backup API method
database: Export parse_table_directory_name() helper
database: Introduce format_table_directory_name() helper
snapshot-ctl: Add config to snapshot_ctl
snapshot-ctl: Add sstables::storage_manager dependency
snapshot-ctl: Maintain task manager module
snapshot-ctl: Add "snapshots" logger
snapshot-ctl: Outline stop() method and constructor
snapshot-ctl: Inline run_snapshot_list<>
test/cql_test_env: Export task manager from cql test env
task_manager: Print task ttl on start (for debugging)
docs: Update object_storage.md with AWS_ environment
docs: Restructure object_storage.md
Increase pool size changes were recently reverted because of the flakiness for the test_gossip_boot test. Test started
to fail on adding the node to the cluster without any issues in the Scylla log file. In test logs it looked like the
installation process for the new node just hanged. After investigating the problem, I've found out that the issue is that
test.py was draining the io_executor pool for cleaning the directory during install that was set to eight workers. So
to fix the issue, io_executor pool should be increased to more or less the same ratio as it was: doubled cluster pool size.
Closesscylladb/scylladb#20276
The `keyspace_compaction` method incorrectly appends the column family
parameter to the URL using a regular string, `"?cf={table}"`, instead of
an f-string, `f"?cf={table}"`. As a result, the column family name is
sent as `{table}` to the server, causing the compaction request to fail.
Fix this issue by passing the parameter to the POST request using a
dictionary instead of appending it to the URL.
Fixes#20264
Signed-off-by: Lakshmi Narayanan Sreethar <lakshmi.sreethar@scylladb.com>
Closesscylladb/scylladb#20243
Namely:
- POST /storage_service/snapshots to take snapshot on a ks
- GET /task_manager/get_task_status/{id} to get status of a running task
- GET /task_manager/wait_task/{id} to wait for a task to finish
- POST /task_manager/abort_task/{id} to abort a running task
Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
The method starts a task that uploads all files from the given
keyspace's snapshot to the requested endpoint/bucket. The task runs in
the background, its task_id is returned from the method once it's
spawned and it should be used via /task_manager API to track the task
execution and completion (hint: it's good to have non-zero TTL value to
make sure fast backups don't finish before the caller manages to call
wait_task API).
If snapshot doesn't exist, nothing happens (FIXME, need to return back
an error in that case).
If endpoint is not configured locally, the API call resolves with
bad-request instantly.
Sstables components are scanned for all tables in the keyspace and are
uploaded into the /bucket/${cf_name}/${snapshot_name}/ path.
Task is not abortable (FIXME -- to be added) and doesn't really report
its progress other than running/done state (FIXME -- to be added too).
Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
The manager.driver_connect() functions allows to pass parameters when
creating the connection (e.g., a special auth_provider), but unfortunately
right now the servers_add() function always calls driver_connect()
without parameters. So in this patch we just add a new optional
parameter to servers_add(), driver_connect_opts, that will be passed to
driver_connect().
In theory instead of the new option to driver_connect() a caller can
pass start=False to servers_add() and later call driver_connect()
manually with the right arguments. The problem is that start=False
avoids more than just calling driver_connect(), so it doesn't solve
the problem.
An example of using the new option is to run Scylla with authentication
enabled, and then connect to it using the correct default account
("cassandra"/"cassandra"):
config = {
'authenticator': 'PasswordAuthenticator',
'authorizer': 'CassandraAuthorizer'
}
servers = await manager.servers_add(1, config=config,
driver_connect_opts={'auth_provider':
PlainTextAuthProvider(username='cassandra', password='cassandra')})
Signed-off-by: Nadav Har'El <nyh@scylladb.com>
Add more logging for raft-based topology operations in INFO and DEBUG
levels.
Improve the existing logging, adding more details.
Fix a FIXME in test_coordinator_queue_management (by readding a log
message that was removed in the past -- probably by accident -- and
properly awaiting for it to appear in test).
Enable group0_state_machine logging at TRACE level in tests. These logs
are relatively rare (group 0 commands are used for metadata operations)
and relatively small, mostly consist of printing `system.group0_history`
mutation in the applied command, for example:
```
TRACE 2024-08-02 18:47:12,238 [shard 0: gms] group0_raft_sm - apply() is called with 1 commands
TRACE 2024-08-02 18:47:12,238 [shard 0: gms] group0_raft_sm - cmd: prev_state_id: optional(dd9d47c6-50ee-11ef-d77f-500b8e1edde3), new_state_id: dd9ea5c6-50ee-11ef-ae64-dfbcd08d72c3, creator_addr: 127.219.233.1, creator_id: 02679305-b9d1-41ef-866d-d69be156c981
TRACE 2024-08-02 18:47:12,238 [shard 0: gms] group0_raft_sm - cmd.history_append: {canonical_mutation: table_id 027e42f5-683a-3ed7-b404-a0100762063c schema_version c9c345e1-428f-36e0-b7d5-9af5f985021e partition_key pk{0007686973746f7279} partition_tombstone {tombstone: none}, row tombstone {range_tombstone: start={position: clustered, ckp{0010b4ba65c64b6e11ef8080808080808080}, 1}, end={position: clustered, ckp{}, 1}, {tombstone: timestamp=1722617232237511, deletion_time=1722617232}}{row {position: clustered, ckp{0010dd9ea5c650ee11efae64dfbcd08d72c3}, 0} tombstone {row_tombstone: none} marker {row_marker: 1722617232237511 0 0}, column description atomic_cell{ create system_distributed keyspace; create system_distributed_everywhere keyspace; create and update system_distributed(_everywhere) tables,ts=1722617232237511,expiry=-1,ttl=0}}}
```
note that the mutation contains a human-readable description of the
command -- like "create system_distributed keyspace" above.
These logs might help debugging various issues (e.g. when `apply` hangs
waiting for read_apply mutex, or takes too long to apply a command).
Ref: scylladb/scylladb#19105
Ref: scylladb/scylladb#19945Closesscylladb/scylladb#19998
Introduce virtual tasks - task manager tasks which cover
cluster-wide operations.
Virtual tasks aren't kept in memory, instead their statuses
are retrieved from associated service when user requests
them with task manager API. From API users' perspective,
virtual tasks behave similarly to regular tasks, but they can
be queried from any node in a cluster.
Virtual tasks cannot have a parent task. They can have
children on each node in a cluster, but do not keep references
to them. So, if a direct child of a virtual task is unregistered
from task manager, it will no longer be shown in parent's
children vector.
virtual_task class corresponds to all virtual tasks in one
group. If users want to list all tasks in a module, a virtual_task
returns all recent supported operations; if they request virtual
task's status - info about the one specified operation is
presented. Time to live, number of tracked operations etc.
depend on the implementation of individual virtual_task.
All virtual_tasks are kept only on shard 0.
Refs: https://github.com/scylladb/scylladb/issues/15852
New feature, no backport needed.
Closesscylladb/scylladb#16374
* github.com:scylladb/scylladb:
docs: describe virtual tasks
db: node_ops: filter topology request entries
test: add a topology suite for testing tasks
node_ops: service: create streaming tasks
node_ops: register node_ops_virtual_task in task manager
service: node_ops: keep node ops module in storage service
node_ops: implement node_ops_virtual_task methods
db: service: modify methods to get topology_requests data
db: service: add request type column to topology_requests
node_ops: add task manager module and node_ops_virtual_task
tasks: api: add virtual task support to get_task_status_recursively
tasks: api: add virtual task support
tasks: api: add virtual tasks support to get_tasks
tasks: add task_handler to hide task and virtual_task differences from user
tasks: modify invoke_on_task
tasks: implement task_manager::virtual_task::impl::get_children
tasks: keep virtual tasks in task manager
tasks: introduce task_manager::virtual_task
We make two changes:
- we lease the IP address of a node that failed to boot because of
an expected error,
- we don't log "Cluster ... added ..." when a node fails to boot
because of an expected error.
Here are some examples of tests that don't work with no initial
nodes, but they should work:
1.
```
await manager.server_add(expected_error="...")
await manager.server_add()
```
2.
```
await manager.servers_add(2, expected_error="...")
await manager.servers_add(2)
```
3.
```
s1 = await manager.server_add(start=False)
await manager.server_start(s1.server_id, expected_error="...")
await manager.server_add()
```
4.
```
[s1, s2] = await manager.servers_add(2, start=False)
await manager.server_start(s1.server_id, expected_error="...")
await manager.server_start(s2.server_id, expected_error="...")
await manager.servers_add(2)
```
5.
```
s1 = await manager.server_add(start=False)
await manager.server_add()
await manager.server_start(s1.server_id)
```
6.
```
[s1, s2] = await manager.servers_add(2, start=False)
await manager.servers_add(2)
await manager.server_start(s1.server_id)
await manager.server_start(s2.server_id)
```
In this patch, we make a few improvements to make tests like the ones
presented above work. I tested all the examples above manually.
From now on, servers receive correct seeds if the first servers added
in the test didn't start or failed to boot.
Also, we remove the assertion preventing the creation of a second
cluster. This assertion failed the tests presented above. We could
weaken it to make these tests pass, but it would require some work.
Moreover, we have tests that intentionally create two clusters.
Therefore, we go for the easiest solution and accept that a single
`ScyllaCluster` may not correspond to a single Scylla cluster.
We change seeds in `ScyllaCluster.server_start` to all currently
running nodes. The previous code only pretended that it did it.
After doing this change, writing tests that create multiple clusters
is impossible. To allow it, we add the `seeds` parameter to
`ManagerClient.server_start`. We use it to fix and simplify the only
test that creates two clusters - `test_different_group0_ids`.
Add topology_tasks test suite for testing task manager's node ops
tasks. Add TaskManagerClient to topology_tasks for an easy usage
of task manager rest api.
Write a test for bootstrap, replace, rebuild, decommission and remove
top level tasks using the above.
Replaced the old `read_barrier` helper from "test/pylib/util.py"
by the new helper from "test/pylib/rest_client.py" that is calling
the newly introduced direct REST API.
Replaced in all relevant tests and decommissioned the old helper.
Introduced a new helper `get_host_api_address` to retrieve the host API
address - which in come cases can be different from the host address
(e.g. if the RPC address is changed).
Fixes: scylladb/scylladb#19662Closesscylladb/scylladb#19739
This will allow to trigger the read barrier directly via the API,
instead of doing work-arounds (like dropping a non-existent table).
The intended use-case is in the Scylla Manager, to make sure that
the database schema is up to date after the data has been backed up
and before attempting to backup the database schema.
The database schema in particular is being backed up just on a single
node, which might not yet have the schema at least as new as the data
(data can be migrated to a newer schema, but not a vice-versa).
The read barrier issued on the node should ensure that the node should
have the schema at least as new as the data or newer.
Closes#19213
This series is another approach of https://github.com/scylladb/scylladb/pull/18646 and https://github.com/scylladb/scylladb/pull/19181. In this series we only change where the view backlog gets
updated - we do not assure that the view update backlog returned in a response is necessarily the backlog
that increased due to the corresponding write, the returned backlog may be outdated up to 10ms. Because
this series does not include this change, it's considerably less complex and it doesn't modify the common
write patch, so no particular performance considerations were needed in that context. The issue being fixed
is still the same, the full description can be seen below.
When a replica applies a write on a table which has a materialized view
it generates view updates. These updates take memory which is tracked
by `database::_view_update_concurrency_sem`, separate on each shard.
The fraction of units taken from the semaphore to the semaphore limit
is the shard's view update backlog. Based on these backlogs, we want
to estimate how busy a node is with its view updates work. We do that
by taking the max backlog across all shards.
To avoid excessive cross-shard operations, the node's (max) backlog isn't
calculated each time we need it, but up to 1 time per 10ms (the `_interval`) with an optimization where the backlog of the calculating shard is immediately up-to-date (we don't need cross-shard operations for it):
```
update_backlog node_update_backlog::fetch() {
auto now = clock::now();
if (now >= _last_update.load(std::memory_order_relaxed) + _interval) {
_last_update.store(now, std::memory_order_relaxed);
auto new_max = boost::accumulate(
_backlogs,
update_backlog::no_backlog(),
[] (const update_backlog& lhs, const per_shard_backlog& rhs) {
return std::max(lhs, rhs.load());
});
_max.store(new_max, std::memory_order_relaxed);
return new_max;
}
return std::max(fetch_shard(this_shard_id()), _max.load(std::memory_order_relaxed));
}
```
For the same reason, even when we do calculate the new node's backlog,
we don't read from the `_view_update_concurrency_sem`. Instead, for
each shard we also store a update_backlog atomic which we use for
calculation:
```
struct per_shard_backlog {
// Multiply by 2 to defeat the prefetcher
alignas(seastar::cache_line_size * 2) std::atomic<update_backlog> backlog = update_backlog::no_backlog();
need_publishing need_publishing = need_publishing::no;
update_backlog load() const {
return backlog.load(std::memory_order_relaxed);
}
};
std::vector<per_shard_backlog> _backlogs;
```
Due to this distinction, the update_backlog atomic need to be updated
separately, when the `_view_update_concurrency_sem` changes.
This is done by calling `storage_proxy::update_view_update_backlog`, which reads the `_view_update_concurrency_sem` of the shard (in `database::get_view_update_backlog`)
and then calls node`_update_backlog::add` where the read backlog
is stored in the atomic:
```
void storage_proxy::update_view_update_backlog() {
_max_view_update_backlog.add(get_db().local().get_view_update_backlog());
}
void node_update_backlog::add(update_backlog backlog) {
_backlogs[this_shard_id()].backlog.store(backlog, std::memory_order_relaxed);
_backlogs[this_shard_id()].need_publishing = need_publishing::yes;
}
```
For this implementation of calculating the node's view update backlog to work,
we need the atomics to be updated correctly when the semaphores of corresponding
shards change.
The main event where the view update backlog changes is an incoming write
request. That's why when handling the request and preparing a response
we update the backlog calling `storage_proxy::get_view_update_backlog` (also
because we want to read the backlog and send it in the response):
backlog update after local view updates (`storage_proxy::send_to_live_endpoints` in `mutate_begin`)
```
auto lmutate = [handler_ptr, response_id, this, my_address, timeout] () mutable {
return handler_ptr->apply_locally(timeout, handler_ptr->get_trace_state())
.then([response_id, this, my_address, h = std::move(handler_ptr), p = shared_from_this()] {
// make mutation alive until it is processed locally, otherwise it
// may disappear if write timeouts before this future is ready
got_response(response_id, my_address, get_view_update_backlog());
});
};
backlog update after remote view updates (storage_proxy::remote::handle_write)
auto f = co_await coroutine::as_future(send_mutation_done(netw::messaging_service::msg_addr{reply_to, shard}, trace_state_ptr,
shard, response_id, p->get_view_update_backlog()));
```
Now assume that on a certain node we have a write request received on shard A,
which updates a row on shard B (A!=B). As a result, shard B will generate view
updates and consume units from its `_view_update_concurrency_sem`, but will
not update its atomic in `_backlogs` yet. Because both shards in the example
are on the same node, shard A will perform a local write calling `lmutate` shown
above. In the `lmutate` call, the `apply_locally` will initiate the actual write on
shard B and the `storage_proxy::update_view_update_backlog` will be called back
on shard A. In no place will the backlog atomic on shard B get updated even
though it increased in size due to the view updates generated there.
Currently, what we calculate there doesn't really matter - it's only used for the
MV flow control delays, so currently, in this scenario, we may only overload
a replica causing failed replica writes which will be later retried as hints. However,
when we add MV admission control, the calculated backlog will be the difference
between an accepted and a rejected request.
Fixes: https://github.com/scylladb/scylladb/issues/18542
Without admission control (https://github.com/scylladb/scylladb/pull/18334), this patch doesn't affect much, so I'm marking it as backport/none
Closesscylladb/scylladb#19341
* github.com:scylladb/scylladb:
test: add test for view backlog not being updated on correct shard
test: move auxiliary methods for waiting until a view is built to util
mv: update view update backlog when it increases on correct shard
In many materialized view tests we need to wait until a view is built before
actually working on it, future tests will also need it. In existing tests
we use the same, duplicated method for achieving that.
In this patch the method is deduplicated and moved to pylib/util.py
and existing tests are modified to use it instead.
this PR has 2 commits
- [test: pass Scylla extra CMD args from test.py args](6b367a04b5)
- [test: adjust scylla_cluster.merge_cmdline_options behavior](c60b36090a)
the main goal is to solve [test.py: provide an easy-to-remember, univeral way to run scylla with trace level logging](https://github.com/scylladb/scylladb/issues/14960) issue
but also can be used to easily apply additional arguments for all UnitTests and PythonTests on the fly from the test.py CMD
Closesscylladb/scylladb#19509
* github.com:scylladb/scylladb:
test: adjust scylla_cluster.merge_cmdline_options behavior
test: pass scylla extra CMD args from test.py args
adjust merge_cmdline_options behaviour to
append --logger-log-level option instead of merge
this behaviour can be changed(if needed)
to previour version(all merge):
merge_cmdline_options(list1, list2, appending_options=[])
or, to append different cmd options:
merge_cmdline_options(list1, list2, appending_options=[option1,option2])
before this change, when running test like:
```console
./test.py --mode release topology_experimental_raft/test_tablets
/home/kefu/dev/scylladb/test/pylib/scylla_cluster.py:333: SyntaxWarning: invalid escape sequence '\('
deleted_sstable_re = f"^.*/{keyspace}/{table}-[0-9a-f]{{32}}/.* \(deleted\)$"
```
we could have the warning above. because `\(` is not a valid escape
sequence, but the Python interpreter accepts it as two separated
characters of `\(` after complaining. but it's still annoying.
so, let's use a raw string here, as we want to match "(deleted)".
Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>
Closesscylladb/scylladb#19451
Related: https://github.com/scylladb/scylladb/issues/17851
Fix the issue that test logs were not deleted
Fix the issue that the URL to the failed test directory was incorrectly shown even when artifacts_dir_url option was not provided
Fix the issue that there were no node logs when it failed to join the cluster
Closesscylladb/scylladb#19115
* github.com:scylladb/scylladb:
[test.py] Fix logs had multiplication of lines
[test.py] Fix log not deleted
[test.py] Fix log for failed node was nod added to failed directory
[test.py] Fix URl for failed logs directory in CI
One of the created log files was not deleted at all, because there was no delete command. Unlink moved on later stage explicitly after removing the handler that writing to this file to avoid the possibility that something will be added after removing the file.
in 51c53d8db6, we check `self.old_env[env]` for None, but there
are chances `self.old_env` does not contain a value with `env`.
in that case, we'd have following failure:
```
Traceback (most recent call last):
File "/home/kefu/dev/scylladb/test/pylib/minio_server.py", line 307, in <module>
asyncio.run(main())
File "/usr/lib64/python3.12/asyncio/runners.py", line 194, in run
return runner.run(main)
^^^^^^^^^^^^^^^^
File "/usr/lib64/python3.12/asyncio/runners.py", line 118, in run
return self._loop.run_until_complete(task)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/lib64/python3.12/asyncio/base_events.py", line 687, in run_until_complete
return future.result()
^^^^^^^^^^^^^^^
File "/home/kefu/dev/scylladb/test/pylib/minio_server.py", line 304, in main
await server.stop()
File "/home/kefu/dev/scylladb/test/pylib/minio_server.py", line 274, in stop
self._unset_environ()
File "/home/kefu/dev/scylladb/test/pylib/minio_server.py", line 211, in _unset_environ
if self.old_env[env] is not None:
~~~~~~~~~~~~^^^^^
KeyError: 'S3_CONFFILE_FOR_TEST'
```
this happens if we run `pylib/minio_server.py` as a standalone
application.
in this change, instead of getting the value with index, we use
`dict.get()`, so that it does not throw when the dict does not
have the given key.
Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>
Closesscylladb/scylladb#19291
If something happens during nod adding to the cluster, it will not be registered as a part of the cluster. This leads to situations during log gathering that logs for a such node will be missing.
In CI test always executed with option --repeat=3 that leads to generate 3 test results with the same name. Junit plugin in CI cannot distinguish correctly the difference between these results. In case when we have two passes and one fail, the link to test result will sometimes be redirected to the incorrect one because the test name is the same. To fix this ReportPlugin added that will be responsible to modify the test case name during junit report generation adding to the test name mode and run id.
Fixes: https://github.com/scylladb/scylladb/issues/17851
Fixes: https://github.com/scylladb/scylladb/issues/15973Closesscylladb/scylladb#19235
* github.com:scylladb/scylladb:
[test.py] Add uniqueness to the test name
[test.py] Refactor alternator, nodetool, rest_api
This PR fixes two problems with the `expected_error`
parameter in `server_add` and `servers_add`.
1. It didn't work in `server_add` if the cluster was empty
because of an incorrect attempt to connect the driver.
2. It didn't work in `servers_add` completely because the
`seeds` parameter was handled incorrectly.
This PR only adds improvements in the testing framework,
no need to backport it.
Closesscylladb/scylladb#19255
* github.com:scylladb/scylladb:
test: manager_client, scylla_cluster: fix type annotations in add_servers
test: manager_client: don't connect driver after failed server_{add, start}
test: scylla_cluster: pass seeds to add_servers
In CI test always executed with option --repeat=3 that leads to generate 3 test results with the same name. Junit plugin in CI cannot distinguish correctly the difference between these results. In case when we have two passes and one fail, the link to test result will sometimes be redirected to the incorrect one because the test name is the same.
To fix this ReportPlugin added that will be responsible to modify the test case name during junit report generation adding to the test name mode and run id.
Fixes: https://github.com/scylladb/scylladb/issues/17851
Fixes: https://github.com/scylladb/scylladb/issues/15973
Before work on tablets was completed, it was noticed that — due to some missing pieces of implementation — Scylla doesn't properly close sstables for migrated-away tablets. Because of this, disk space wasn't being reclaimed properly.
Since the missing pieces of implementation were added, the problem should be gone now. This patch adds a test which was used to reproduce the problem earlier. It's expected to pass now, validating that the issue was fixed.
Should be backported to branch-6.0, because the tested problem was also affecting that branch.
Fixes#16946Closesscylladb/scylladb#18906
* github.com:scylladb/scylladb:
test_tablets: add test_tablet_storage_freeing
test: pylib: add get_sstables_disk_usage()
If adding or starting a server fails expectedly, there is no reason
to update or connect the driver. Moreover, before this patch, we
couldn't use `server_add` and `servers_add` with `expected_error`
if the cluster was empty. After expected bootstrap failures, we
tried to connect the driver, which rightfully failed on
`assert len(hosts) > 0` in `cluster_con`.
This parameter was incorrectly missing. For this reason,
`expected_error` was passed from `add_servers` to `add_server` as
`seeds`, which caused strange crashes.
This config item is propagated to the table object via table::config. Although the field in `table::config`, used to propagate the value, was `utils::updateable_value<T>`, it was assigned a constant and so the live-update chain was broken.
This series fixes this and adds a test which fails before the patch and passes after. The test needed new test infrastructure, around the failure injection api, namely the ability to exfiltrate the value of internal variable. This infrastructure is also added in this series.
Fixes: https://github.com/scylladb/scylladb/issues/18674
- [x] This patch has to be backported because it fixes broken functionality
Closesscylladb/scylladb#18705
* github.com:scylladb/scylladb:
test/topology_custom: add test for enable_compacting_data_for_streaming_and_repair live-update
test/pylib: rest_client: add get_injection()
api/error_injection: add getter for error_injection
utils/error_injection: add set_parameter()
replica/database: fix live-update enable_compacting_data_for_streaming_and_repair
Consider the following:
1) table A has N tablets and views
2) migration starts for a tablet of A from node 1 to 2.
3) migration is at write_both_read_old stage
4) coordinator will push writes to both nodes (pending and leaving)
5) A has view, so writes to it will also result in reads (table::push_view_replica_updates())
6) tablet's update_effective_replication_map() is not refreshing tablet sstable set (for new tablet migrating in)
7) so read on step 5 is not being able to find sstable set for tablet migrating in
Causes the following error:
"tablets - SSTable set wasn't found for tablet 21 of table mview.users"
which means loss of write on pending replica.
The fix will refresh the table's sstable set (tablet_sstable_set) and cache's snapshot.
It's not a problem to refresh the cache snapshot as long as the logical
state of the data hasn't changed, which is true when allocating new
tablet replicas. That's also done in the context of compactions for example.
Fixes#19052.
Fixes#19033.
Signed-off-by: Raphael S. Carvalho <raphaelsc@scylladb.com>
Closesscylladb/scylladb#19099